diff --git a/.gitignore b/.gitignore index 6e09e097456e..9f8cd0b4cb23 100644 --- a/.gitignore +++ b/.gitignore @@ -72,6 +72,7 @@ metastore/ metastore_db/ sql/hive-thriftserver/test_warehouses warehouse/ +spark-warehouse/ # For R session data .RData diff --git a/LICENSE b/LICENSE index 9714b3b1e4d1..f40364046b34 100644 --- a/LICENSE +++ b/LICENSE @@ -263,7 +263,7 @@ The text of each license is also included at licenses/LICENSE-[project].txt. (New BSD license) Protocol Buffer Java API (org.spark-project.protobuf:protobuf-java:2.4.1-shaded - http://code.google.com/p/protobuf) (The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net) (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net) - (The New BSD License) Py4J (net.sf.py4j:py4j:0.9.2 - http://py4j.sourceforge.net/) + (The New BSD License) Py4J (net.sf.py4j:py4j:0.10.1 - http://py4j.sourceforge.net/) (Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/) (BSD licence) sbt and sbt-launch-lib.bash (BSD 3 Clause) d3.min.js (https://github.com/mbostock/d3/blob/master/LICENSE) diff --git a/NOTICE b/NOTICE index 2a6fe237dcbe..f4b1260a2217 100644 --- a/NOTICE +++ b/NOTICE @@ -12,7 +12,9 @@ Common Development and Distribution License 1.0 The following components are provided under the Common Development and Distribution License 1.0. See project link for details. (CDDL 1.0) Glassfish Jasper (org.mortbay.jetty:jsp-2.1:6.1.14 - http://jetty.mortbay.org/project/modules/jsp-2.1) + (CDDL 1.0) JAX-RS (https://jax-rs-spec.java.net/) (CDDL 1.0) Servlet Specification 2.5 API (org.mortbay.jetty:servlet-api-2.5:6.1.14 - http://jetty.mortbay.org/project/modules/servlet-api-2.5) + (CDDL 1.0) (GPL2 w/ CPE) javax.annotation API (https://glassfish.java.net/nonav/public/CDDL+GPL.html) (COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0) (GNU General Public Library) Streaming API for XML (javax.xml.stream:stax-api:1.0-2 - no url defined) (Common Development and Distribution License (CDDL) v1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp) @@ -22,15 +24,10 @@ Common Development and Distribution License 1.1 The following components are provided under the Common Development and Distribution License 1.1. See project link for details. + (CDDL 1.1) (GPL2 w/ CPE) org.glassfish.hk2 (https://hk2.java.net) (CDDL 1.1) (GPL2 w/ CPE) JAXB API bundle for GlassFish V3 (javax.xml.bind:jaxb-api:2.2.2 - https://jaxb.dev.java.net/) (CDDL 1.1) (GPL2 w/ CPE) JAXB RI (com.sun.xml.bind:jaxb-impl:2.2.3-1 - http://jaxb.java.net/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:1.8 - https://jersey.dev.java.net/jersey-core/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:1.9 - https://jersey.java.net/jersey-core/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-guice (com.sun.jersey.contribs:jersey-guice:1.9 - https://jersey.java.net/jersey-contribs/jersey-guice/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:1.8 - https://jersey.dev.java.net/jersey-json/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:1.9 - https://jersey.java.net/jersey-json/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:1.8 - https://jersey.dev.java.net/jersey-server/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:1.9 - https://jersey.java.net/jersey-server/) + (CDDL 1.1) (GPL2 w/ CPE) Jersey 2 (https://jersey.java.net) ======================================================================== Common Public License 1.0 diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 7179438efc1d..963a1bb5806a 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -26,6 +26,7 @@ Collate: 'pairRDD.R' 'DataFrame.R' 'SQLContext.R' + 'WindowSpec.R' 'backend.R' 'broadcast.R' 'client.R' @@ -38,4 +39,5 @@ Collate: 'stats.R' 'types.R' 'utils.R' + 'window.R' RoxygenNote: 5.0.1 diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 73f7c595f443..239ad065d09a 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -47,6 +47,7 @@ exportMethods("arrange", "covar_pop", "crosstab", "dapply", + "dapplyCollect", "describe", "dim", "distinct", @@ -216,6 +217,7 @@ exportMethods("%in%", "next_day", "ntile", "otherwise", + "over", "percent_rank", "pmod", "quarter", @@ -315,3 +317,12 @@ export("structField", "structType.jobj", "structType.structField", "print.structType") + +exportClasses("WindowSpec") + +export("partitionBy", + "rowsBetween", + "rangeBetween") + +export("window.partitionBy", + "window.orderBy") diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 9e30fa0dbf26..0c2a194483b0 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -570,10 +570,17 @@ setMethod("unpersist", #' Repartition #' -#' Return a new SparkDataFrame that has exactly numPartitions partitions. -#' +#' The following options for repartition are possible: +#' \itemize{ +#' \item{"Option 1"} {Return a new SparkDataFrame partitioned by +#' the given columns into `numPartitions`.} +#' \item{"Option 2"} {Return a new SparkDataFrame that has exactly `numPartitions`.} +#' \item{"Option 3"} {Return a new SparkDataFrame partitioned by the given column(s), +#' using `spark.sql.shuffle.partitions` as number of partitions.} +#'} #' @param x A SparkDataFrame #' @param numPartitions The number of partitions to use. +#' @param col The column by which the partitioning will be performed. #' #' @family SparkDataFrame functions #' @rdname repartition @@ -586,11 +593,31 @@ setMethod("unpersist", #' path <- "path/to/file.json" #' df <- read.json(sqlContext, path) #' newDF <- repartition(df, 2L) +#' newDF <- repartition(df, numPartitions = 2L) +#' newDF <- repartition(df, col = df$"col1", df$"col2") +#' newDF <- repartition(df, 3L, col = df$"col1", df$"col2") #'} setMethod("repartition", - signature(x = "SparkDataFrame", numPartitions = "numeric"), - function(x, numPartitions) { - sdf <- callJMethod(x@sdf, "repartition", numToInt(numPartitions)) + signature(x = "SparkDataFrame"), + function(x, numPartitions = NULL, col = NULL, ...) { + if (!is.null(numPartitions) && is.numeric(numPartitions)) { + # number of partitions and columns both are specified + if (!is.null(col) && class(col) == "Column") { + cols <- list(col, ...) + jcol <- lapply(cols, function(c) { c@jc }) + sdf <- callJMethod(x@sdf, "repartition", numToInt(numPartitions), jcol) + } else { + # only number of partitions is specified + sdf <- callJMethod(x@sdf, "repartition", numToInt(numPartitions)) + } + } else if (!is.null(col) && class(col) == "Column") { + # only columns are specified + cols <- list(col, ...) + jcol <- lapply(cols, function(c) { c@jc }) + sdf <- callJMethod(x@sdf, "repartition", jcol) + } else { + stop("Please, specify the number of partitions and/or a column(s)") + } dataFrame(sdf) }) @@ -1126,9 +1153,27 @@ setMethod("summarize", agg(x, ...) }) +dapplyInternal <- function(x, func, schema) { + packageNamesArr <- serialize(.sparkREnv[[".packages"]], + connection = NULL) + + broadcastArr <- lapply(ls(.broadcastNames), + function(name) { get(name, .broadcastNames) }) + + sdf <- callJStatic( + "org.apache.spark.sql.api.r.SQLUtils", + "dapply", + x@sdf, + serialize(cleanClosure(func), connection = NULL), + packageNamesArr, + broadcastArr, + if (is.null(schema)) { schema } else { schema$jobj }) + dataFrame(sdf) +} + #' dapply #' -#' Apply a function to each partition of a DataFrame. +#' Apply a function to each partition of a SparkDataFrame. #' #' @param x A SparkDataFrame #' @param func A function to be applied to each partition of the SparkDataFrame. @@ -1170,21 +1215,57 @@ setMethod("summarize", setMethod("dapply", signature(x = "SparkDataFrame", func = "function", schema = "structType"), function(x, func, schema) { - packageNamesArr <- serialize(.sparkREnv[[".packages"]], - connection = NULL) - - broadcastArr <- lapply(ls(.broadcastNames), - function(name) { get(name, .broadcastNames) }) - - sdf <- callJStatic( - "org.apache.spark.sql.api.r.SQLUtils", - "dapply", - x@sdf, - serialize(cleanClosure(func), connection = NULL), - packageNamesArr, - broadcastArr, - schema$jobj) - dataFrame(sdf) + dapplyInternal(x, func, schema) + }) + +#' dapplyCollect +#' +#' Apply a function to each partition of a SparkDataFrame and collect the result back +#’ to R as a data.frame. +#' +#' @param x A SparkDataFrame +#' @param func A function to be applied to each partition of the SparkDataFrame. +#' func should have only one parameter, to which a data.frame corresponds +#' to each partition will be passed. +#' The output of func should be a data.frame. +#' @family SparkDataFrame functions +#' @rdname dapply +#' @name dapplyCollect +#' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame (sqlContext, iris) +#' ldf <- dapplyCollect(df, function(x) { x }) +#' +#' # filter and add a column +#' df <- createDataFrame ( +#' sqlContext, +#' list(list(1L, 1, "1"), list(2L, 2, "2"), list(3L, 3, "3")), +#' c("a", "b", "c")) +#' ldf <- dapplyCollect( +#' df, +#' function(x) { +#' y <- x[x[1] > 1, ] +#' y <- cbind(y, y[1] + 1L) +#' }) +#' # the result +#' # a b c d +#' # 2 2 2 3 +#' # 3 3 3 4 +#' } +setMethod("dapplyCollect", + signature(x = "SparkDataFrame", func = "function"), + function(x, func) { + df <- dapplyInternal(x, func, NULL) + + content <- callJMethod(df@sdf, "collect") + # content is a list of items of struct type. Each item has a single field + # which is a serialized data.frame corresponds to one partition of the + # SparkDataFrame. + ldfs <- lapply(content, function(x) { unserialize(x[[1]]) }) + ldf <- do.call(rbind, ldfs) + row.names(ldf) <- NULL + ldf }) ############################## RDD Map Functions ################################## @@ -1722,8 +1803,8 @@ setMethod("arrange", #' @export setMethod("orderBy", signature(x = "SparkDataFrame", col = "characterOrColumn"), - function(x, col) { - arrange(x, col) + function(x, col, ...) { + arrange(x, col, ...) }) #' Filter diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 34d29ddbfdd5..f1badf4364da 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -1023,9 +1023,13 @@ setMethod("keyBy", #' @aliases repartition,RDD #' @noRd setMethod("repartition", - signature(x = "RDD", numPartitions = "numeric"), + signature(x = "RDD"), function(x, numPartitions) { - coalesce(x, numPartitions, TRUE) + if (!is.null(numPartitions) && is.numeric(numPartitions)) { + coalesce(x, numPartitions, TRUE) + } else { + stop("Please, specify the number of partitions") + } }) #' Return a new RDD that is reduced into numPartitions partitions. diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 3824e0a99557..6b7a341bee88 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -298,6 +298,8 @@ parquetFile <- function(sqlContext, ...) { #' Create a SparkDataFrame from a text file. #' #' Loads a text file and returns a SparkDataFrame with a single string column named "value". +#' If the directory structure of the text files contains partitioning information, those are +#' ignored in the resulting DataFrame. #' Each line in the text file is a new row in the resulting SparkDataFrame. #' #' @param sqlContext SQLContext to use diff --git a/R/pkg/R/WindowSpec.R b/R/pkg/R/WindowSpec.R new file mode 100644 index 000000000000..581176a6c091 --- /dev/null +++ b/R/pkg/R/WindowSpec.R @@ -0,0 +1,188 @@ +# +# 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. +# + +# WindowSpec.R - WindowSpec class and methods implemented in S4 OO classes + +#' @include generics.R jobj.R column.R +NULL + +#' @title S4 class that represents a WindowSpec +#' @description WindowSpec can be created by using window.partitionBy() +#' or window.orderBy() +#' @rdname WindowSpec +#' @seealso \link{window.partitionBy}, \link{window.orderBy} +#' +#' @param sws A Java object reference to the backing Scala WindowSpec +#' @export +setClass("WindowSpec", + slots = list(sws = "jobj")) + +setMethod("initialize", "WindowSpec", function(.Object, sws) { + .Object@sws <- sws + .Object +}) + +windowSpec <- function(sws) { + stopifnot(class(sws) == "jobj") + new("WindowSpec", sws) +} + +#' @rdname show +setMethod("show", "WindowSpec", + function(object) { + cat("WindowSpec", callJMethod(object@sws, "toString"), "\n") + }) + +#' partitionBy +#' +#' Defines the partitioning columns in a WindowSpec. +#' +#' @param x a WindowSpec +#' @return a WindowSpec +#' @rdname partitionBy +#' @name partitionBy +#' @family windowspec_method +#' @export +#' @examples +#' \dontrun{ +#' partitionBy(ws, "col1", "col2") +#' partitionBy(ws, df$col1, df$col2) +#' } +setMethod("partitionBy", + signature(x = "WindowSpec"), + function(x, col, ...) { + stopifnot (class(col) %in% c("character", "Column")) + + if (class(col) == "character") { + windowSpec(callJMethod(x@sws, "partitionBy", col, list(...))) + } else { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec(callJMethod(x@sws, "partitionBy", jcols)) + } + }) + +#' orderBy +#' +#' Defines the ordering columns in a WindowSpec. +#' +#' @param x a WindowSpec +#' @return a WindowSpec +#' @rdname arrange +#' @name orderBy +#' @family windowspec_method +#' @export +#' @examples +#' \dontrun{ +#' orderBy(ws, "col1", "col2") +#' orderBy(ws, df$col1, df$col2) +#' } +setMethod("orderBy", + signature(x = "WindowSpec", col = "character"), + function(x, col, ...) { + windowSpec(callJMethod(x@sws, "orderBy", col, list(...))) + }) + +#' @rdname arrange +#' @name orderBy +#' @export +setMethod("orderBy", + signature(x = "WindowSpec", col = "Column"), + function(x, col, ...) { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec(callJMethod(x@sws, "orderBy", jcols)) + }) + +#' rowsBetween +#' +#' Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). +#' +#' Both `start` and `end` are relative positions from the current row. For example, "0" means +#' "current row", while "-1" means the row before the current row, and "5" means the fifth row +#' after the current row. +#' +#' @param x a WindowSpec +#' @param start boundary start, inclusive. +#' The frame is unbounded if this is the minimum long value. +#' @param end boundary end, inclusive. +#' The frame is unbounded if this is the maximum long value. +#' @return a WindowSpec +#' @rdname rowsBetween +#' @name rowsBetween +#' @family windowspec_method +#' @export +#' @examples +#' \dontrun{ +#' rowsBetween(ws, 0, 3) +#' } +setMethod("rowsBetween", + signature(x = "WindowSpec", start = "numeric", end = "numeric"), + function(x, start, end) { + # "start" and "end" should be long, due to serde limitation, + # limit "start" and "end" as integer now + windowSpec(callJMethod(x@sws, "rowsBetween", as.integer(start), as.integer(end))) + }) + +#' rangeBetween +#' +#' Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). +#' +#' Both `start` and `end` are relative from the current row. For example, "0" means "current row", +#' while "-1" means one off before the current row, and "5" means the five off after the +#' current row. + +#' @param x a WindowSpec +#' @param start boundary start, inclusive. +#' The frame is unbounded if this is the minimum long value. +#' @param end boundary end, inclusive. +#' The frame is unbounded if this is the maximum long value. +#' @return a WindowSpec +#' @rdname rangeBetween +#' @name rangeBetween +#' @family windowspec_method +#' @export +#' @examples +#' \dontrun{ +#' rangeBetween(ws, 0, 3) +#' } +setMethod("rangeBetween", + signature(x = "WindowSpec", start = "numeric", end = "numeric"), + function(x, start, end) { + # "start" and "end" should be long, due to serde limitation, + # limit "start" and "end" as integer now + windowSpec(callJMethod(x@sws, "rangeBetween", as.integer(start), as.integer(end))) + }) + +# Note that over is a method of Column class, but it is placed here to +# avoid Roxygen circular-dependency between class Column and WindowSpec. + +#' over +#' +#' Define a windowing column. +#' +#' @rdname over +#' @name over +#' @family colum_func +#' @export +setMethod("over", + signature(x = "Column", window = "WindowSpec"), + function(x, window) { + column(callJMethod(x@jc, "over", window@sws)) + }) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index f936ea603998..ed76ad6b73c8 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -167,7 +167,7 @@ setGeneric("reduce", function(x, func) { standardGeneric("reduce") }) # @rdname repartition # @seealso coalesce # @export -setGeneric("repartition", function(x, numPartitions) { standardGeneric("repartition") }) +setGeneric("repartition", function(x, ...) { standardGeneric("repartition") }) # @rdname sampleRDD # @export @@ -339,9 +339,9 @@ setGeneric("join", function(x, y, ...) { standardGeneric("join") }) # @export setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) -# @rdname partitionBy -# @export -setGeneric("partitionBy", function(x, numPartitions, ...) { standardGeneric("partitionBy") }) +#' @rdname partitionBy +#' @export +setGeneric("partitionBy", function(x, ...) { standardGeneric("partitionBy") }) # @rdname reduceByKey # @seealso groupByKey @@ -450,6 +450,10 @@ setGeneric("covar_pop", function(col1, col2) {standardGeneric("covar_pop") }) #' @export setGeneric("dapply", function(x, func, schema) { standardGeneric("dapply") }) +#' @rdname dapply +#' @export +setGeneric("dapplyCollect", function(x, func) { standardGeneric("dapplyCollect") }) + #' @rdname summary #' @export setGeneric("describe", function(x, col, ...) { standardGeneric("describe") }) @@ -533,7 +537,7 @@ setGeneric("mutate", function(.data, ...) {standardGeneric("mutate") }) #' @rdname arrange #' @export -setGeneric("orderBy", function(x, col) { standardGeneric("orderBy") }) +setGeneric("orderBy", function(x, col, ...) { standardGeneric("orderBy") }) #' @rdname schema #' @export @@ -733,6 +737,27 @@ setGeneric("when", function(condition, value) { standardGeneric("when") }) #' @export setGeneric("otherwise", function(x, value) { standardGeneric("otherwise") }) +#' @rdname over +#' @export +setGeneric("over", function(x, window) { standardGeneric("over") }) + +###################### WindowSpec Methods ########################## + +#' @rdname rowsBetween +#' @export +setGeneric("rowsBetween", function(x, start, end) { standardGeneric("rowsBetween") }) + +#' @rdname rangeBetween +#' @export +setGeneric("rangeBetween", function(x, start, end) { standardGeneric("rangeBetween") }) + +#' @rdname window.partitionBy +#' @export +setGeneric("window.partitionBy", function(col, ...) { standardGeneric("window.partitionBy") }) + +#' @rdname window.orderBy +#' @export +setGeneric("window.orderBy", function(col, ...) { standardGeneric("window.orderBy") }) ###################### Expression Function Methods ########################## diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 4075ef4377ac..d39775cabef8 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -205,8 +205,10 @@ setMethod("flatMapValues", #' @aliases partitionBy,RDD,integer-method #' @noRd setMethod("partitionBy", - signature(x = "RDD", numPartitions = "numeric"), + signature(x = "RDD"), function(x, numPartitions, partitionFunc = hashCode) { + stopifnot(is.numeric(numPartitions)) + partitionFunc <- cleanClosure(partitionFunc) serializedHashFuncBytes <- serialize(partitionFunc, connection = NULL) diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index c187869fdf12..04a8b1e1f395 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -103,8 +103,7 @@ sparkR.stop <- function() { #' list(spark.executor.memory="4g"), #' list(LD_LIBRARY_PATH="/directory of JVM libraries (libjvm.so) on workers/"), #' c("one.jar", "two.jar", "three.jar"), -#' c("com.databricks:spark-avro_2.10:2.0.1", -#' "com.databricks:spark-csv_2.10:1.3.0")) +#' c("com.databricks:spark-avro_2.10:2.0.1")) #'} sparkR.init <- function( diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index bf67e231d56f..784f7371807e 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -157,8 +157,11 @@ wrapInt <- function(value) { # Multiply `val` by 31 and add `addVal` to the result. Ensures that # integer-overflows are handled at every step. +# +# TODO: this function does not handle integer overflow well mult31AndAdd <- function(val, addVal) { vec <- c(bitwShiftL(val, c(4, 3, 2, 1, 0)), addVal) + vec[is.na(vec)] <- 0 Reduce(function(a, b) { wrapInt(as.numeric(a) + as.numeric(b)) }, diff --git a/R/pkg/R/window.R b/R/pkg/R/window.R new file mode 100644 index 000000000000..7ecf70abc6b5 --- /dev/null +++ b/R/pkg/R/window.R @@ -0,0 +1,98 @@ +# +# 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. +# + +# window.R - Utility functions for defining window in DataFrames + +#' window.partitionBy +#' +#' Creates a WindowSpec with the partitioning defined. +#' +#' @rdname window.partitionBy +#' @name window.partitionBy +#' @export +#' @examples +#' \dontrun{ +#' ws <- window.partitionBy("key1", "key2") +#' df1 <- select(df, over(lead("value", 1), ws)) +#' +#' ws <- window.partitionBy(df$key1, df$key2) +#' df1 <- select(df, over(lead("value", 1), ws)) +#' } +setMethod("window.partitionBy", + signature(col = "character"), + function(col, ...) { + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "partitionBy", + col, + list(...))) + }) + +#' @rdname window.partitionBy +#' @name window.partitionBy +#' @export +setMethod("window.partitionBy", + signature(col = "Column"), + function(col, ...) { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "partitionBy", + jcols)) + }) + +#' window.orderBy +#' +#' Creates a WindowSpec with the ordering defined. +#' +#' @rdname window.orderBy +#' @name window.orderBy +#' @export +#' @examples +#' \dontrun{ +#' ws <- window.orderBy("key1", "key2") +#' df1 <- select(df, over(lead("value", 1), ws)) +#' +#' ws <- window.orderBy(df$key1, df$key2) +#' df1 <- select(df, over(lead("value", 1), ws)) +#' } +setMethod("window.orderBy", + signature(col = "character"), + function(col, ...) { + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "orderBy", + col, + list(...))) + }) + +#' @rdname window.orderBy +#' @name window.orderBy +#' @export +setMethod("window.orderBy", + signature(col = "Column"), + function(col, ...) { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "orderBy", + jcols)) + }) diff --git a/R/pkg/inst/tests/testthat/test_client.R b/R/pkg/inst/tests/testthat/test_client.R index a0664f32f31c..0cf25fe1dbf3 100644 --- a/R/pkg/inst/tests/testthat/test_client.R +++ b/R/pkg/inst/tests/testthat/test_client.R @@ -32,14 +32,12 @@ test_that("no package specified doesn't add packages flag", { }) test_that("multiple packages don't produce a warning", { - expect_that(generateSparkSubmitArgs("", "", "", "", c("A", "B")), not(gives_warning())) + expect_warning(generateSparkSubmitArgs("", "", "", "", c("A", "B")), NA) }) test_that("sparkJars sparkPackages as character vectors", { args <- generateSparkSubmitArgs("", "", c("one.jar", "two.jar", "three.jar"), "", - c("com.databricks:spark-avro_2.10:2.0.1", - "com.databricks:spark-csv_2.10:1.3.0")) + c("com.databricks:spark-avro_2.10:2.0.1")) expect_match(args, "--jars one.jar,two.jar,three.jar") - expect_match(args, - "--packages com.databricks:spark-avro_2.10:2.0.1,com.databricks:spark-csv_2.10:1.3.0") + expect_match(args, "--packages com.databricks:spark-avro_2.10:2.0.1") }) diff --git a/R/pkg/inst/tests/testthat/test_context.R b/R/pkg/inst/tests/testthat/test_context.R index ca04342cd512..0e5e15c0a96c 100644 --- a/R/pkg/inst/tests/testthat/test_context.R +++ b/R/pkg/inst/tests/testthat/test_context.R @@ -138,7 +138,7 @@ test_that("sparkJars sparkPackages as comma-separated strings", { # check normalizePath f <- dir()[[1]] - expect_that(processSparkJars(f), not(gives_warning())) + expect_warning(processSparkJars(f), NA) expect_match(processSparkJars(f), f) }) diff --git a/R/pkg/inst/tests/testthat/test_mllib.R b/R/pkg/inst/tests/testthat/test_mllib.R index 37d87aa8a046..5f8a27d4e094 100644 --- a/R/pkg/inst/tests/testthat/test_mllib.R +++ b/R/pkg/inst/tests/testthat/test_mllib.R @@ -450,9 +450,9 @@ test_that("spark.survreg", { if (requireNamespace("survival", quietly = TRUE)) { rData <- list(time = c(4, 3, 1, 1, 2, 2, 3), status = c(1, 1, 1, 0, 1, 1, 0), x = c(0, 2, 1, 1, 1, 0, 0), sex = c(0, 0, 0, 0, 1, 1, 1)) - expect_that( + expect_error( model <- survival::survreg(formula = survival::Surv(time, status) ~ x + sex, data = rData), - not(throws_error())) + NA) expect_equal(predict(model, rData)[[1]], 3.724591, tolerance = 1e-4) } }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 5cf9dc405b16..6a99b43e5aa5 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1196,9 +1196,9 @@ test_that("date functions on a DataFrame", { c(as.POSIXlt("2012-12-13 21:34:00 UTC"), as.POSIXlt("2014-12-15 10:24:34 UTC"))) expect_equal(collect(select(df2, to_utc_timestamp(df2$b, "JST")))[, 1], c(as.POSIXlt("2012-12-13 03:34:00 UTC"), as.POSIXlt("2014-12-14 16:24:34 UTC"))) - expect_more_than(collect(select(df2, unix_timestamp()))[1, 1], 0) - expect_more_than(collect(select(df2, unix_timestamp(df2$b)))[1, 1], 0) - expect_more_than(collect(select(df2, unix_timestamp(lit("2015-01-01"), "yyyy-MM-dd")))[1, 1], 0) + 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) l3 <- list(list(a = 1000), list(a = -1000)) df3 <- createDataFrame(sqlContext, l3) @@ -1502,7 +1502,6 @@ test_that("toJSON() returns an RDD of the correct values", { test_that("showDF()", { df <- read.json(sqlContext, jsonPath) - s <- capture.output(showDF(df)) expected <- paste("+----+-------+\n", "| age| name|\n", "+----+-------+\n", @@ -1510,7 +1509,7 @@ test_that("showDF()", { "| 30| Andy|\n", "| 19| Justin|\n", "+----+-------+\n", sep = "") - expect_output(s, expected) + expect_output(showDF(df), expected) }) test_that("isLocal()", { @@ -1896,7 +1895,7 @@ test_that("Method as.data.frame as a synonym for collect()", { expect_equal(as.data.frame(irisDF2), collect(irisDF2)) # Make sure as.data.frame in the R base package is not covered - expect_that(as.data.frame(c(1, 2)), not(throws_error())) + expect_error(as.data.frame(c(1, 2)), NA) }) test_that("attach() on a DataFrame", { @@ -2044,7 +2043,7 @@ test_that("Histogram", { expect_equal(histogram(df, "x")$counts, c(4, 0, 0, 0, 0, 0, 0, 0, 0, 1)) }) -test_that("dapply() on a DataFrame", { +test_that("dapply() and dapplyCollect() on a DataFrame", { df <- createDataFrame ( sqlContext, list(list(1L, 1, "1"), list(2L, 2, "2"), list(3L, 3, "3")), @@ -2054,6 +2053,8 @@ test_that("dapply() on a DataFrame", { result <- collect(df1) expect_identical(ldf, result) + result <- dapplyCollect(df, function(x) { x }) + expect_identical(ldf, result) # Filter and add a column schema <- structType(structField("a", "integer"), structField("b", "double"), @@ -2071,6 +2072,16 @@ test_that("dapply() on a DataFrame", { rownames(expected) <- NULL expect_identical(expected, result) + result <- dapplyCollect( + df, + function(x) { + y <- x[x$a > 1, ] + y <- cbind(y, y$a + 1L) + }) + expected1 <- expected + names(expected1) <- names(result) + expect_identical(expected1, result) + # Remove the added column df2 <- dapply( df1, @@ -2081,6 +2092,85 @@ test_that("dapply() on a DataFrame", { result <- collect(df2) expected <- expected[, c("a", "b", "c")] expect_identical(expected, result) + + result <- dapplyCollect( + df1, + function(x) { + x[, c("a", "b", "c")] + }) + expect_identical(expected, result) +}) + +test_that("repartition by columns on DataFrame", { + df <- createDataFrame ( + sqlContext, + list(list(1L, 1, "1", 0.1), list(1L, 2, "2", 0.2), list(3L, 3, "3", 0.3)), + c("a", "b", "c", "d")) + + # no column and number of partitions specified + retError <- tryCatch(repartition(df), error = function(e) e) + expect_equal(grepl + ("Please, specify the number of partitions and/or a column\\(s\\)", retError), TRUE) + + # repartition by column and number of partitions + actual <- repartition(df, 3L, col = df$"a") + + # since we cannot access the number of partitions from dataframe, checking + # that at least the dimensions are identical + expect_identical(dim(df), dim(actual)) + + # repartition by number of partitions + actual <- repartition(df, 13L) + expect_identical(dim(df), dim(actual)) + + # a test case with a column and dapply + schema <- structType(structField("a", "integer"), structField("avg", "double")) + df <- repartition(df, col = df$"a") + df1 <- dapply( + df, + function(x) { + y <- (data.frame(x$a[1], mean(x$b))) + }, + schema) + + # Number of partitions is equal to 2 + expect_equal(nrow(df1), 2) +}) + +test_that("Window functions on a DataFrame", { + ssc <- callJMethod(sc, "sc") + hiveCtx <- tryCatch({ + newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc) + }, + error = function(err) { + skip("Hive is not build with SparkSQL, skipped") + }) + + df <- createDataFrame(hiveCtx, + list(list(1L, "1"), list(2L, "2"), list(1L, "1"), list(2L, "2")), + schema = c("key", "value")) + ws <- orderBy(window.partitionBy("key"), "value") + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expected <- data.frame(key = c(1L, NA, 2L, NA), + value = c("1", NA, "2", NA), + stringsAsFactors = FALSE) + expect_equal(result, expected) + + ws <- orderBy(window.partitionBy(df$key), df$value) + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expect_equal(result, expected) + + ws <- partitionBy(window.orderBy("value"), "key") + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expect_equal(result, expected) + + ws <- partitionBy(window.orderBy(df$value), df$key) + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expect_equal(result, expected) }) unlink(parquetPath) diff --git a/R/pkg/inst/tests/testthat/test_utils.R b/R/pkg/inst/tests/testthat/test_utils.R index 01694ab5c4f6..54d2eca50eaf 100644 --- a/R/pkg/inst/tests/testthat/test_utils.R +++ b/R/pkg/inst/tests/testthat/test_utils.R @@ -164,3 +164,7 @@ test_that("convertToJSaveMode", { expect_error(convertToJSaveMode("foo"), 'mode should be one of "append", "overwrite", "error", "ignore"') #nolint }) + +test_that("hashCode", { + expect_error(hashCode("bc53d3605e8a5b7de1e8e271c2317645"), NA) +}) diff --git a/bin/pyspark b/bin/pyspark index d1fe75a08bda..396a07c9f413 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -63,7 +63,7 @@ export PYSPARK_PYTHON # Add the PySpark classes to the Python path: export PYTHONPATH="${SPARK_HOME}/python/:$PYTHONPATH" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.9.2-src.zip:$PYTHONPATH" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.1-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index cb788497ffc7..3e2ff100fb8a 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -30,7 +30,7 @@ if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( ) set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH% -set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.9.2-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py diff --git a/bin/spark-class b/bin/spark-class index b2a36b984678..23a60c6ee44f 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -64,8 +64,25 @@ fi # The launcher library will print arguments separated by a NULL character, to allow arguments with # characters that would be otherwise interpreted by the shell. Read that in a while loop, populating # an array that will be used to exec the final command. +# +# The exit code of the launcher is appended to the output, so the parent shell removes it from the +# command array and checks the value to see if the launcher succeeded. +build_command() { + "$RUNNER" -cp "$LAUNCH_CLASSPATH" org.apache.spark.launcher.Main "$@" + printf "%d\0" $? +} + CMD=() while IFS= read -d '' -r ARG; do CMD+=("$ARG") -done < <("$RUNNER" -cp "$LAUNCH_CLASSPATH" org.apache.spark.launcher.Main "$@") +done < <(build_command "$@") + +COUNT=${#CMD[@]} +LAST=$((COUNT - 1)) +LAUNCHER_EXIT_CODE=${CMD[$LAST]} +if [ $LAUNCHER_EXIT_CODE != 0 ]; then + exit $LAUNCHER_EXIT_CODE +fi + +CMD=("${CMD[@]:0:$LAST}") exec "${CMD[@]}" diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index bd507c2cb6c4..5444ae6d70a9 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -66,7 +66,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} org.mockito diff --git a/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java index 844eff4f4c70..c20fab83c346 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java @@ -130,7 +130,7 @@ public ManagedBuffer release() { @Override public Object convertToNetty() throws IOException { if (conf.lazyFileDescriptor()) { - return new LazyFileRegion(file, offset, length); + return new DefaultFileRegion(file, offset, length); } else { FileChannel fileChannel = new FileInputStream(file).getChannel(); return new DefaultFileRegion(fileChannel, offset, length); diff --git a/common/network-common/src/main/java/org/apache/spark/network/buffer/LazyFileRegion.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/LazyFileRegion.java deleted file mode 100644 index 162cf6da0dff..000000000000 --- a/common/network-common/src/main/java/org/apache/spark/network/buffer/LazyFileRegion.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * 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.network.buffer; - -import java.io.FileInputStream; -import java.io.File; -import java.io.IOException; -import java.nio.channels.FileChannel; -import java.nio.channels.WritableByteChannel; - -import com.google.common.base.Objects; -import io.netty.channel.FileRegion; -import io.netty.util.AbstractReferenceCounted; - -import org.apache.spark.network.util.JavaUtils; - -/** - * A FileRegion implementation that only creates the file descriptor when the region is being - * transferred. This cannot be used with Epoll because there is no native support for it. - * - * This is mostly copied from DefaultFileRegion implementation in Netty. In the future, we - * should push this into Netty so the native Epoll transport can support this feature. - */ -public final class LazyFileRegion extends AbstractReferenceCounted implements FileRegion { - - private final File file; - private final long position; - private final long count; - - private FileChannel channel; - - private long numBytesTransferred = 0L; - - /** - * @param file file to transfer. - * @param position start position for the transfer. - * @param count number of bytes to transfer starting from position. - */ - public LazyFileRegion(File file, long position, long count) { - this.file = file; - this.position = position; - this.count = count; - } - - @Override - protected void deallocate() { - JavaUtils.closeQuietly(channel); - } - - @Override - public long position() { - return position; - } - - @Override - public long transfered() { - return numBytesTransferred; - } - - @Override - public long count() { - return count; - } - - @Override - public long transferTo(WritableByteChannel target, long position) throws IOException { - if (channel == null) { - channel = new FileInputStream(file).getChannel(); - } - - long count = this.count - position; - if (count < 0 || position < 0) { - throw new IllegalArgumentException( - "position out of range: " + position + " (expected: 0 - " + (count - 1) + ')'); - } - - if (count == 0) { - return 0L; - } - - long written = channel.transferTo(this.position + position, count, target); - if (written > 0) { - numBytesTransferred += written; - } - return written; - } - - @Override - public String toString() { - return Objects.toStringHelper(this) - .add("file", file) - .add("position", position) - .add("count", count) - .toString(); - } -} diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 810ec10ca05b..e736436aec4c 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -80,7 +80,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} log4j 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 f8d03b3b9433..22fd592a321d 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 @@ -35,6 +35,7 @@ import org.apache.spark.network.server.StreamManager; import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId; import org.apache.spark.network.shuffle.protocol.*; +import org.apache.spark.network.util.NettyUtils; import org.apache.spark.network.util.TransportConf; @@ -86,7 +87,11 @@ protected void handleMessage( blocks.add(blockManager.getBlockData(msg.appId, msg.execId, blockId)); } long streamId = streamManager.registerStream(client.getClientId(), blocks.iterator()); - logger.trace("Registered streamId {} with {} buffers", streamId, msg.blockIds.length); + logger.trace("Registered streamId {} with {} buffers for client {} from host {}", + streamId, + msg.blockIds.length, + client.getClientId(), + NettyUtils.getRemoteAddress(client.getChannel())); callback.onSuccess(new StreamHandle(streamId, msg.blockIds.length).toByteBuffer()); } else if (msgObj instanceof RegisterExecutor) { diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index bc83ef24c30e..1fd3af2e6e62 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -48,7 +48,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 8bc1f5279894..bbbb0bd5aa05 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -38,7 +38,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 8e702b4fefe8..14e94eca93b2 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -27,12 +27,12 @@ org.apache.spark - spark-test-tags_2.11 + spark-tags_2.11 jar - Spark Project Test Tags + Spark Project Tags http://spark.apache.org/ - test-tags + tags diff --git a/core/src/main/java/org/apache/spark/annotation/AlphaComponent.java b/common/tags/src/main/java/org/apache/spark/annotation/AlphaComponent.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/AlphaComponent.java rename to common/tags/src/main/java/org/apache/spark/annotation/AlphaComponent.java diff --git a/core/src/main/java/org/apache/spark/annotation/DeveloperApi.java b/common/tags/src/main/java/org/apache/spark/annotation/DeveloperApi.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/DeveloperApi.java rename to common/tags/src/main/java/org/apache/spark/annotation/DeveloperApi.java diff --git a/core/src/main/java/org/apache/spark/annotation/Experimental.java b/common/tags/src/main/java/org/apache/spark/annotation/Experimental.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/Experimental.java rename to common/tags/src/main/java/org/apache/spark/annotation/Experimental.java diff --git a/core/src/main/java/org/apache/spark/annotation/Private.java b/common/tags/src/main/java/org/apache/spark/annotation/Private.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/Private.java rename to common/tags/src/main/java/org/apache/spark/annotation/Private.java diff --git a/core/src/main/scala/org/apache/spark/annotation/Since.scala b/common/tags/src/main/scala/org/apache/spark/annotation/Since.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/annotation/Since.scala rename to common/tags/src/main/scala/org/apache/spark/annotation/Since.scala diff --git a/core/src/main/scala/org/apache/spark/annotation/package-info.java b/common/tags/src/main/scala/org/apache/spark/annotation/package-info.java similarity index 100% rename from core/src/main/scala/org/apache/spark/annotation/package-info.java rename to common/tags/src/main/scala/org/apache/spark/annotation/package-info.java diff --git a/core/src/main/scala/org/apache/spark/annotation/package.scala b/common/tags/src/main/scala/org/apache/spark/annotation/package.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/annotation/package.scala rename to common/tags/src/main/scala/org/apache/spark/annotation/package.scala diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 93b9580f26b8..d0d1da69ea80 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -36,6 +36,10 @@ + + org.apache.spark + spark-tags_${scala.binary.version} + com.twitter chill_${scala.binary.version} @@ -59,10 +63,6 @@ - - org.apache.spark - spark-test-tags_${scala.binary.version} - org.mockito mockito-core diff --git a/core/pom.xml b/core/pom.xml index 7349ad35b959..45f8bfcc05aa 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -125,12 +125,15 @@ jetty-servlet compile - - org.eclipse.jetty.orbit - javax.servlet - ${orbit.version} + org.eclipse.jetty + jetty-servlets + compile + + + javax.servlet + javax.servlet-api + ${javaxservlet.version} @@ -194,12 +197,24 @@ json4s-jackson_${scala.binary.version} - com.sun.jersey + org.glassfish.jersey.core + jersey-client + + + org.glassfish.jersey.core + jersey-common + + + org.glassfish.jersey.core jersey-server - com.sun.jersey - jersey-core + org.glassfish.jersey.containers + jersey-container-servlet + + + org.glassfish.jersey.containers + jersey-container-servlet-core org.apache.mesos @@ -260,12 +275,11 @@ org.seleniumhq.selenium selenium-java - - - com.google.guava - guava - - + test + + + org.seleniumhq.selenium + selenium-htmlunit-driver test @@ -313,11 +327,11 @@ net.sf.py4j py4j - 0.9.2 + 0.10.1 org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} @@ -344,7 +358,7 @@ true true - guava,jetty-io,jetty-servlet,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server,jetty-security + guava,jetty-io,jetty-servlet,jetty-servlets,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server,jetty-security true diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index 840f13b39464..38a21a896e1f 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -31,15 +31,24 @@ public abstract class MemoryConsumer { protected final TaskMemoryManager taskMemoryManager; private final long pageSize; + private final MemoryMode mode; protected long used; - protected MemoryConsumer(TaskMemoryManager taskMemoryManager, long pageSize) { + protected MemoryConsumer(TaskMemoryManager taskMemoryManager, long pageSize, MemoryMode mode) { this.taskMemoryManager = taskMemoryManager; this.pageSize = pageSize; + this.mode = mode; } protected MemoryConsumer(TaskMemoryManager taskMemoryManager) { - this(taskMemoryManager, taskMemoryManager.pageSizeBytes()); + this(taskMemoryManager, taskMemoryManager.pageSizeBytes(), MemoryMode.ON_HEAP); + } + + /** + * Returns the memory mode, ON_HEAP or OFF_HEAP. + */ + public MemoryMode getMode() { + return mode; } /** @@ -132,19 +141,19 @@ protected void freePage(MemoryBlock page) { } /** - * Allocates a heap memory of `size`. + * Allocates memory of `size`. */ - public long acquireOnHeapMemory(long size) { - long granted = taskMemoryManager.acquireExecutionMemory(size, MemoryMode.ON_HEAP, this); + public long acquireMemory(long size) { + long granted = taskMemoryManager.acquireExecutionMemory(size, this); used += granted; return granted; } /** - * Release N bytes of heap memory. + * Release N bytes of memory. */ - public void freeOnHeapMemory(long size) { - taskMemoryManager.releaseExecutionMemory(size, MemoryMode.ON_HEAP, this); + public void freeMemory(long size) { + taskMemoryManager.releaseExecutionMemory(size, this); used -= size; } } 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 2796114fc545..a4a571f15a8c 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -76,9 +76,6 @@ public class TaskMemoryManager { /** Bit mask for the lower 51 bits of a long. */ private static final long MASK_LONG_LOWER_51_BITS = 0x7FFFFFFFFFFFFL; - /** Bit mask for the upper 13 bits of a long */ - private static final long MASK_LONG_UPPER_13_BITS = ~MASK_LONG_LOWER_51_BITS; - /** * 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 @@ -132,11 +129,10 @@ public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) { * * @return number of bytes successfully granted (<= N). */ - public long acquireExecutionMemory( - long required, - MemoryMode mode, - MemoryConsumer consumer) { + public long acquireExecutionMemory(long required, MemoryConsumer consumer) { assert(required >= 0); + assert(consumer != null); + MemoryMode mode = consumer.getMode(); // If we are allocating Tungsten pages off-heap and receive a request to allocate on-heap // memory here, then it may not make sense to spill since that would only end up freeing // off-heap memory. This is subject to change, though, so it may be risky to make this @@ -149,10 +145,10 @@ public long acquireExecutionMemory( if (got < required) { // Call spill() on other consumers to release memory for (MemoryConsumer c: consumers) { - if (c != consumer && c.getUsed() > 0) { + if (c != consumer && c.getUsed() > 0 && c.getMode() == mode) { try { long released = c.spill(required - got, consumer); - if (released > 0 && mode == tungstenMemoryMode) { + if (released > 0) { logger.debug("Task {} released {} from {} for {}", taskAttemptId, Utils.bytesToString(released), c, consumer); got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId, mode); @@ -170,10 +166,10 @@ public long acquireExecutionMemory( } // call spill() on itself - if (got < required && consumer != null) { + if (got < required) { try { long released = consumer.spill(required - got, consumer); - if (released > 0 && mode == tungstenMemoryMode) { + if (released > 0) { logger.debug("Task {} released {} from itself ({})", taskAttemptId, Utils.bytesToString(released), consumer); got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId, mode); @@ -185,9 +181,7 @@ public long acquireExecutionMemory( } } - if (consumer != null) { - consumers.add(consumer); - } + consumers.add(consumer); logger.debug("Task {} acquire {} for {}", taskAttemptId, Utils.bytesToString(got), consumer); return got; } @@ -196,9 +190,9 @@ public long acquireExecutionMemory( /** * Release N bytes of execution memory for a MemoryConsumer. */ - public void releaseExecutionMemory(long size, MemoryMode mode, MemoryConsumer consumer) { + public void releaseExecutionMemory(long size, MemoryConsumer consumer) { logger.debug("Task {} release {} from {}", taskAttemptId, Utils.bytesToString(size), consumer); - memoryManager.releaseExecutionMemory(size, taskAttemptId, mode); + memoryManager.releaseExecutionMemory(size, taskAttemptId, consumer.getMode()); } /** @@ -241,12 +235,14 @@ public long pageSizeBytes() { * contains fewer bytes than requested, so callers should verify the size of returned pages. */ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { + assert(consumer != null); + assert(consumer.getMode() == tungstenMemoryMode); if (size > MAXIMUM_PAGE_SIZE_BYTES) { throw new IllegalArgumentException( "Cannot allocate a page with more than " + MAXIMUM_PAGE_SIZE_BYTES + " bytes"); } - long acquired = acquireExecutionMemory(size, tungstenMemoryMode, consumer); + long acquired = acquireExecutionMemory(size, consumer); if (acquired <= 0) { return null; } @@ -255,7 +251,7 @@ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { synchronized (this) { pageNumber = allocatedPages.nextClearBit(0); if (pageNumber >= PAGE_TABLE_SIZE) { - releaseExecutionMemory(acquired, tungstenMemoryMode, consumer); + releaseExecutionMemory(acquired, consumer); throw new IllegalStateException( "Have already allocated a maximum of " + PAGE_TABLE_SIZE + " pages"); } @@ -299,7 +295,7 @@ public void freePage(MemoryBlock page, MemoryConsumer consumer) { } long pageSize = page.size(); memoryManager.tungstenMemoryAllocator().free(page); - releaseExecutionMemory(pageSize, tungstenMemoryMode, consumer); + releaseExecutionMemory(pageSize, consumer); } /** @@ -379,7 +375,6 @@ public long getOffsetInPage(long pagePlusOffsetAddress) { */ public long cleanUpAllAllocatedMemory() { synchronized (this) { - Arrays.fill(pageTable, null); for (MemoryConsumer c: consumers) { if (c != null && c.getUsed() > 0) { // In case of failed task, it's normal to see leaked memory @@ -387,16 +382,17 @@ public long cleanUpAllAllocatedMemory() { } } consumers.clear(); - } - for (MemoryBlock page : pageTable) { - if (page != null) { - memoryManager.tungstenMemoryAllocator().free(page); + for (MemoryBlock page : pageTable) { + if (page != null) { + logger.warn("leak a page: " + page + " in task " + taskAttemptId); + memoryManager.tungstenMemoryAllocator().free(page); + } } + Arrays.fill(pageTable, null); } - Arrays.fill(pageTable, null); - // release the memory that is not used by any consumer. + // release the memory that is not used by any consumer (acquired for pages in tungsten mode). memoryManager.releaseExecutionMemory(acquiredButNotUsed, taskAttemptId, tungstenMemoryMode); return memoryManager.releaseAllExecutionMemoryForTask(taskAttemptId); @@ -412,7 +408,7 @@ public long getMemoryConsumptionForThisTask() { /** * Returns Tungsten memory mode */ - public MemoryMode getTungstenMemoryMode(){ + public MemoryMode getTungstenMemoryMode() { return tungstenMemoryMode; } } 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 2be5a16b2d1e..014aef86b5cc 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 @@ -104,8 +104,9 @@ final class ShuffleExternalSorter extends MemoryConsumer { int numPartitions, SparkConf conf, ShuffleWriteMetrics writeMetrics) { - super(memoryManager, (int) Math.min(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, - memoryManager.pageSizeBytes())); + super(memoryManager, + (int) Math.min(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, memoryManager.pageSizeBytes()), + memoryManager.getTungstenMemoryMode()); this.taskMemoryManager = memoryManager; this.blockManager = blockManager; this.taskContext = taskContext; 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 6807710f9fef..6c00608302c4 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 @@ -182,7 +182,7 @@ public BytesToBytesMap( double loadFactor, long pageSizeBytes, boolean enablePerfMetrics) { - super(taskMemoryManager, pageSizeBytes); + super(taskMemoryManager, pageSizeBytes, taskMemoryManager.getTungstenMemoryMode()); this.taskMemoryManager = taskMemoryManager; this.blockManager = blockManager; this.serializerManager = serializerManager; diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java index de92b8db4713..e9571aa8bb05 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java @@ -17,7 +17,7 @@ package org.apache.spark.util.collection.unsafe.sort; -final class RecordPointerAndKeyPrefix { +public final class RecordPointerAndKeyPrefix { /** * A pointer to a record; see {@link org.apache.spark.memory.TaskMemoryManager} for a * description of how these addresses are encoded. 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 8b6c96a4c4e6..e14a23f4a6a8 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 @@ -76,6 +76,7 @@ public final class UnsafeExternalSorter extends MemoryConsumer { private long pageCursor = -1; private long peakMemoryUsedBytes = 0; private long totalSpillBytes = 0L; + private long totalSortTimeNanos = 0L; private volatile SpillableIterator readingIterator = null; public static UnsafeExternalSorter createWithExistingInMemorySorter( @@ -123,7 +124,7 @@ private UnsafeExternalSorter( long pageSizeBytes, @Nullable UnsafeInMemorySorter existingInMemorySorter, boolean canUseRadixSort) { - super(taskMemoryManager, pageSizeBytes); + super(taskMemoryManager, pageSizeBytes, taskMemoryManager.getTungstenMemoryMode()); this.taskMemoryManager = taskMemoryManager; this.blockManager = blockManager; this.serializerManager = serializerManager; @@ -247,6 +248,17 @@ public long getPeakMemoryUsedBytes() { return peakMemoryUsedBytes; } + /** + * @return the total amount of time spent sorting data (in-memory only). + */ + public long getSortTimeNanos() { + UnsafeInMemorySorter sorter = inMemSorter; + if (sorter != null) { + return sorter.getSortTimeNanos(); + } + return totalSortTimeNanos; + } + /** * Return the total number of bytes that has been spilled into disk so far. */ @@ -505,6 +517,7 @@ public long spill() throws IOException { // in-memory sorter will not be used after spilling assert(inMemSorter != null); released += inMemSorter.getMemoryUsage(); + totalSortTimeNanos += inMemSorter.getSortTimeNanos(); inMemSorter.free(); inMemSorter = null; taskContext.taskMetrics().incMemoryBytesSpilled(released); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 03973f3c124e..0cce792f33d3 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -97,6 +97,8 @@ public int compare(RecordPointerAndKeyPrefix r1, RecordPointerAndKeyPrefix r2) { private long initialSize; + private long totalSortTimeNanos = 0L; + public UnsafeInMemorySorter( final MemoryConsumer consumer, final TaskMemoryManager memoryManager, @@ -160,6 +162,13 @@ public int numRecords() { return pos / 2; } + /** + * @return the total amount of time spent sorting data (in-memory only). + */ + public long getSortTimeNanos() { + return totalSortTimeNanos; + } + public long getMemoryUsage() { return array.size() * 8; } @@ -265,6 +274,7 @@ public void loadNext() { */ public SortedIterator getSortedIterator() { int offset = 0; + long start = System.nanoTime(); if (sorter != null) { if (this.radixSortSupport != null) { // TODO(ekl) we should handle NULL values before radix sort for efficiency, since they @@ -275,6 +285,7 @@ public SortedIterator getSortedIterator() { sorter.sort(array, 0, pos / 2, sortComparator); } } + totalSortTimeNanos += System.nanoTime() - start; return new SortedIterator(pos / 2, offset); } } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java index 12fb62fb77f0..d19b71fbc1bc 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java @@ -29,7 +29,8 @@ * Within each long[] buffer, position {@code 2 * i} holds a pointer pointer to the record at * index {@code i}, while position {@code 2 * i + 1} in the array holds an 8-byte key prefix. */ -final class UnsafeSortDataFormat extends SortDataFormat { +public final class UnsafeSortDataFormat + extends SortDataFormat { public static final UnsafeSortDataFormat INSTANCE = new UnsafeSortDataFormat(); diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js index f4453c71df1e..9ab5684d901f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js @@ -15,7 +15,7 @@ * limitations under the License. */ -function drawApplicationTimeline(groupArray, eventObjArray, startTime) { +function drawApplicationTimeline(groupArray, eventObjArray, startTime, offset) { var groups = new vis.DataSet(groupArray); var items = new vis.DataSet(eventObjArray); var container = $("#application-timeline")[0]; @@ -26,7 +26,10 @@ function drawApplicationTimeline(groupArray, eventObjArray, startTime) { editable: false, showCurrentTime: false, min: startTime, - zoomable: false + zoomable: false, + moment: function (date) { + return vis.moment(date).utcOffset(offset); + } }; var applicationTimeline = new vis.Timeline(container); @@ -41,7 +44,7 @@ function drawApplicationTimeline(groupArray, eventObjArray, startTime) { $(".item.range.job.application-timeline-object").each(function() { var getSelectorForJobEntry = function(baseElem) { var jobIdText = $($(baseElem).find(".application-timeline-content")[0]).text(); - var jobId = jobIdText.match("\\(Job (\\d+)\\)")[1]; + var jobId = jobIdText.match("\\(Job (\\d+)\\)$")[1]; return "#job-" + jobId; }; @@ -87,7 +90,7 @@ $(function (){ } }); -function drawJobTimeline(groupArray, eventObjArray, startTime) { +function drawJobTimeline(groupArray, eventObjArray, startTime, offset) { var groups = new vis.DataSet(groupArray); var items = new vis.DataSet(eventObjArray); var container = $('#job-timeline')[0]; @@ -99,6 +102,9 @@ function drawJobTimeline(groupArray, eventObjArray, startTime) { showCurrentTime: false, min: startTime, zoomable: false, + moment: function (date) { + return vis.moment(date).utcOffset(offset); + } }; var jobTimeline = new vis.Timeline(container); @@ -113,7 +119,7 @@ function drawJobTimeline(groupArray, eventObjArray, startTime) { $(".item.range.stage.job-timeline-object").each(function() { var getSelectorForStageEntry = function(baseElem) { var stageIdText = $($(baseElem).find(".job-timeline-content")[0]).text(); - var stageIdAndAttempt = stageIdText.match("\\(Stage (\\d+\\.\\d+)\\)")[1].split("."); + var stageIdAndAttempt = stageIdText.match("\\(Stage (\\d+\\.\\d+)\\)$")[1].split("."); return "#stage-" + stageIdAndAttempt[0] + "-" + stageIdAndAttempt[1]; }; @@ -159,7 +165,7 @@ $(function (){ } }); -function drawTaskAssignmentTimeline(groupArray, eventObjArray, minLaunchTime, maxFinishTime) { +function drawTaskAssignmentTimeline(groupArray, eventObjArray, minLaunchTime, maxFinishTime, offset) { var groups = new vis.DataSet(groupArray); var items = new vis.DataSet(eventObjArray); var container = $("#task-assignment-timeline")[0] @@ -173,7 +179,10 @@ function drawTaskAssignmentTimeline(groupArray, eventObjArray, minLaunchTime, ma showCurrentTime: false, min: minLaunchTime, max: maxFinishTime, - zoomable: false + zoomable: false, + moment: function (date) { + return vis.moment(date).utcOffset(offset); + } }; var taskTimeline = new vis.Timeline(container) diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis.min.css b/core/src/main/resources/org/apache/spark/ui/static/vis.min.css index a390c40d6757..40d182cfde23 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/vis.min.css +++ b/core/src/main/resources/org/apache/spark/ui/static/vis.min.css @@ -1 +1 @@ -.vis .overlay{position:absolute;top:0;left:0;width:100%;height:100%;z-index:10}.vis-active{box-shadow:0 0 10px #86d5f8}.vis [class*=span]{min-height:0;width:auto}.vis.timeline.root{position:relative;border:1px solid #bfbfbf;overflow:hidden;padding:0;margin:0;box-sizing:border-box}.vis.timeline .vispanel{position:absolute;padding:0;margin:0;box-sizing:border-box}.vis.timeline .vispanel.bottom,.vis.timeline .vispanel.center,.vis.timeline .vispanel.left,.vis.timeline .vispanel.right,.vis.timeline .vispanel.top{border:1px #bfbfbf}.vis.timeline .vispanel.center,.vis.timeline .vispanel.left,.vis.timeline .vispanel.right{border-top-style:solid;border-bottom-style:solid;overflow:hidden}.vis.timeline .vispanel.bottom,.vis.timeline .vispanel.center,.vis.timeline .vispanel.top{border-left-style:solid;border-right-style:solid}.vis.timeline .background{overflow:hidden}.vis.timeline .vispanel>.content{position:relative}.vis.timeline .vispanel .shadow{position:absolute;width:100%;height:1px;box-shadow:0 0 10px rgba(0,0,0,.8)}.vis.timeline .vispanel .shadow.top{top:-1px;left:0}.vis.timeline .vispanel .shadow.bottom{bottom:-1px;left:0}.vis.timeline .labelset{position:relative;overflow:hidden;box-sizing:border-box}.vis.timeline .labelset .vlabel{position:relative;left:0;top:0;width:100%;color:#4d4d4d;box-sizing:border-box;border-bottom:1px solid #bfbfbf}.vis.timeline .labelset .vlabel:last-child{border-bottom:none}.vis.timeline .labelset .vlabel .inner{display:inline-block;padding:5px}.vis.timeline .labelset .vlabel .inner.hidden{padding:0}.vis.timeline .itemset{position:relative;padding:0;margin:0;box-sizing:border-box}.vis.timeline .itemset .background,.vis.timeline .itemset .foreground{position:absolute;width:100%;height:100%;overflow:visible}.vis.timeline .axis{position:absolute;width:100%;height:0;left:0;z-index:1}.vis.timeline .foreground .group{position:relative;box-sizing:border-box;border-bottom:1px solid #bfbfbf}.vis.timeline .foreground .group:last-child{border-bottom:none}.vis.timeline .item{position:absolute;color:#1A1A1A;border-color:#97B0F8;border-width:1px;background-color:#D5DDF6;display:inline-block;padding:5px}.vis.timeline .item.selected{border-color:#FFC200;background-color:#FFF785;z-index:2}.vis.timeline .editable .item.selected{cursor:move}.vis.timeline .item.point.selected{background-color:#FFF785}.vis.timeline .item.box{text-align:center;border-style:solid;border-radius:2px}.vis.timeline .item.point{background:0 0}.vis.timeline .item.dot{position:absolute;padding:0;border-width:4px;border-style:solid;border-radius:4px}.vis.timeline .item.range{border-style:solid;border-radius:2px;box-sizing:border-box}.vis.timeline .item.background{overflow:hidden;border:none;background-color:rgba(213,221,246,.4);box-sizing:border-box;padding:0;margin:0}.vis.timeline .item.range .content{position:relative;display:inline-block;max-width:100%;overflow:hidden}.vis.timeline .item.background .content{position:absolute;display:inline-block;overflow:hidden;max-width:100%;margin:5px}.vis.timeline .item.line{padding:0;position:absolute;width:0;border-left-width:1px;border-left-style:solid}.vis.timeline .item .content{white-space:nowrap;overflow:hidden}.vis.timeline .item .delete{background:url(img/timeline/delete.png) top center no-repeat;position:absolute;width:24px;height:24px;top:0;right:-24px;cursor:pointer}.vis.timeline .item.range .drag-left{position:absolute;width:24px;height:100%;top:0;left:-4px;cursor:w-resize}.vis.timeline .item.range .drag-right{position:absolute;width:24px;height:100%;top:0;right:-4px;cursor:e-resize}.vis.timeline .timeaxis{position:relative;overflow:hidden}.vis.timeline .timeaxis.foreground{top:0;left:0;width:100%}.vis.timeline .timeaxis.background{position:absolute;top:0;left:0;width:100%;height:100%}.vis.timeline .timeaxis .text{position:absolute;color:#4d4d4d;padding:3px;white-space:nowrap}.vis.timeline .timeaxis .text.measure{position:absolute;padding-left:0;padding-right:0;margin-left:0;margin-right:0;visibility:hidden}.vis.timeline .timeaxis .grid.vertical{position:absolute;border-left:1px solid}.vis.timeline .timeaxis .grid.minor{border-color:#e5e5e5}.vis.timeline .timeaxis .grid.major{border-color:#bfbfbf}.vis.timeline .currenttime{background-color:#FF7F6E;width:2px;z-index:1}.vis.timeline .customtime{background-color:#6E94FF;width:2px;cursor:move;z-index:1}.vis.timeline .vispanel.background.horizontal .grid.horizontal{position:absolute;width:100%;height:0;border-bottom:1px solid}.vis.timeline .vispanel.background.horizontal .grid.minor{border-color:#e5e5e5}.vis.timeline .vispanel.background.horizontal .grid.major{border-color:#bfbfbf}.vis.timeline .dataaxis .yAxis.major{width:100%;position:absolute;color:#4d4d4d;white-space:nowrap}.vis.timeline .dataaxis .yAxis.major.measure{padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis.timeline .dataaxis .yAxis.minor{position:absolute;width:100%;color:#bebebe;white-space:nowrap}.vis.timeline .dataaxis .yAxis.minor.measure{padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis.timeline .dataaxis .yAxis.title{position:absolute;color:#4d4d4d;white-space:nowrap;bottom:20px;text-align:center}.vis.timeline .dataaxis .yAxis.title.measure{padding:0;margin:0;visibility:hidden;width:auto}.vis.timeline .dataaxis .yAxis.title.left{bottom:0;-webkit-transform-origin:left top;-moz-transform-origin:left top;-ms-transform-origin:left top;-o-transform-origin:left top;transform-origin:left bottom;-webkit-transform:rotate(-90deg);-moz-transform:rotate(-90deg);-ms-transform:rotate(-90deg);-o-transform:rotate(-90deg);transform:rotate(-90deg)}.vis.timeline .dataaxis .yAxis.title.right{bottom:0;-webkit-transform-origin:right bottom;-moz-transform-origin:right bottom;-ms-transform-origin:right bottom;-o-transform-origin:right bottom;transform-origin:right bottom;-webkit-transform:rotate(90deg);-moz-transform:rotate(90deg);-ms-transform:rotate(90deg);-o-transform:rotate(90deg);transform:rotate(90deg)}.vis.timeline .legend{background-color:rgba(247,252,255,.65);padding:5px;border-color:#b3b3b3;border-style:solid;border-width:1px;box-shadow:2px 2px 10px rgba(154,154,154,.55)}.vis.timeline .legendText{white-space:nowrap;display:inline-block}.vis.timeline .graphGroup0{fill:#4f81bd;fill-opacity:0;stroke-width:2px;stroke:#4f81bd}.vis.timeline .graphGroup1{fill:#f79646;fill-opacity:0;stroke-width:2px;stroke:#f79646}.vis.timeline .graphGroup2{fill:#8c51cf;fill-opacity:0;stroke-width:2px;stroke:#8c51cf}.vis.timeline .graphGroup3{fill:#75c841;fill-opacity:0;stroke-width:2px;stroke:#75c841}.vis.timeline .graphGroup4{fill:#ff0100;fill-opacity:0;stroke-width:2px;stroke:#ff0100}.vis.timeline .graphGroup5{fill:#37d8e6;fill-opacity:0;stroke-width:2px;stroke:#37d8e6}.vis.timeline .graphGroup6{fill:#042662;fill-opacity:0;stroke-width:2px;stroke:#042662}.vis.timeline .graphGroup7{fill:#00ff26;fill-opacity:0;stroke-width:2px;stroke:#00ff26}.vis.timeline .graphGroup8{fill:#f0f;fill-opacity:0;stroke-width:2px;stroke:#f0f}.vis.timeline .graphGroup9{fill:#8f3938;fill-opacity:0;stroke-width:2px;stroke:#8f3938}.vis.timeline .fill{fill-opacity:.1;stroke:none}.vis.timeline .bar{fill-opacity:.5;stroke-width:1px}.vis.timeline .point{stroke-width:2px;fill-opacity:1}.vis.timeline .legendBackground{stroke-width:1px;fill-opacity:.9;fill:#fff;stroke:#c2c2c2}.vis.timeline .outline{stroke-width:1px;fill-opacity:1;fill:#fff;stroke:#e5e5e5}.vis.timeline .iconFill{fill-opacity:.3;stroke:none}div.network-manipulationDiv{border-width:0;border-bottom:1px;border-style:solid;border-color:#d6d9d8;background:#fff;background:-moz-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#fff),color-stop(48%,#fcfcfc),color-stop(50%,#fafafa),color-stop(100%,#fcfcfc));background:-webkit-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-o-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-ms-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:linear-gradient(to bottom,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffff', endColorstr='#fcfcfc', GradientType=0);position:absolute;left:0;top:0;width:100%;height:30px}div.network-manipulation-editMode{position:absolute;left:0;top:0;height:30px;margin-top:20px}div.network-manipulation-closeDiv{position:absolute;right:0;top:0;width:30px;height:30px;background-position:20px 3px;background-repeat:no-repeat;background-image:url(img/network/cross.png);cursor:pointer;-webkit-touch-callout:none;-webkit-user-select:none;-khtml-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}div.network-manipulation-closeDiv:hover{opacity:.6}span.network-manipulationUI{font-family:verdana;font-size:12px;-moz-border-radius:15px;border-radius:15px;display:inline-block;background-position:0 0;background-repeat:no-repeat;height:24px;margin:-14px 0 0 10px;vertical-align:middle;cursor:pointer;padding:0 8px;-webkit-touch-callout:none;-webkit-user-select:none;-khtml-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}span.network-manipulationUI:hover{box-shadow:1px 1px 8px rgba(0,0,0,.2)}span.network-manipulationUI:active{box-shadow:1px 1px 8px rgba(0,0,0,.5)}span.network-manipulationUI.back{background-image:url(img/network/backIcon.png)}span.network-manipulationUI.none:hover{box-shadow:1px 1px 8px transparent;cursor:default}span.network-manipulationUI.none:active{box-shadow:1px 1px 8px transparent}span.network-manipulationUI.none{padding:0}span.network-manipulationUI.notification{margin:2px;font-weight:700}span.network-manipulationUI.add{background-image:url(img/network/addNodeIcon.png)}span.network-manipulationUI.edit{background-image:url(img/network/editIcon.png)}span.network-manipulationUI.edit.editmode{background-color:#fcfcfc;border-style:solid;border-width:1px;border-color:#ccc}span.network-manipulationUI.connect{background-image:url(img/network/connectIcon.png)}span.network-manipulationUI.delete{background-image:url(img/network/deleteIcon.png)}span.network-manipulationLabel{margin:0 0 0 23px;line-height:25px}div.network-seperatorLine{display:inline-block;width:1px;height:20px;background-color:#bdbdbd;margin:5px 7px 0 15px}div.network-navigation_wrapper{position:absolute;left:0;top:0;width:100%;height:100%}div.network-navigation{width:34px;height:34px;-moz-border-radius:17px;border-radius:17px;position:absolute;display:inline-block;background-position:2px 2px;background-repeat:no-repeat;cursor:pointer;-webkit-touch-callout:none;-webkit-user-select:none;-khtml-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}div.network-navigation:hover{box-shadow:0 0 3px 3px rgba(56,207,21,.3)}div.network-navigation:active{box-shadow:0 0 1px 3px rgba(56,207,21,.95)}div.network-navigation.up{background-image:url(img/network/upArrow.png);bottom:50px;left:55px}div.network-navigation.down{background-image:url(img/network/downArrow.png);bottom:10px;left:55px}div.network-navigation.left{background-image:url(img/network/leftArrow.png);bottom:10px;left:15px}div.network-navigation.right{background-image:url(img/network/rightArrow.png);bottom:10px;left:95px}div.network-navigation.zoomIn{background-image:url(img/network/plus.png);bottom:10px;right:15px}div.network-navigation.zoomOut{background-image:url(img/network/minus.png);bottom:10px;right:55px}div.network-navigation.zoomExtends{background-image:url(img/network/zoomExtends.png);bottom:50px;right:15px} \ No newline at end of file +.vis-background,.vis-labelset,.vis-timeline{overflow:hidden}.vis .overlay{position:absolute;top:0;left:0;width:100%;height:100%;z-index:10}.vis-active{box-shadow:0 0 10px #86d5f8}.vis [class*=span]{min-height:0;width:auto}div.vis-configuration{position:relative;display:block;float:left;font-size:12px}div.vis-configuration-wrapper{display:block;width:700px}div.vis-configuration-wrapper::after{clear:both;content:"";display:block}div.vis-configuration.vis-config-option-container{display:block;width:495px;background-color:#fff;border:2px solid #f7f8fa;border-radius:4px;margin-top:20px;left:10px;padding-left:5px}div.vis-configuration.vis-config-button{display:block;width:495px;height:25px;vertical-align:middle;line-height:25px;background-color:#f7f8fa;border:2px solid #ceced0;border-radius:4px;margin-top:20px;left:10px;padding-left:5px;cursor:pointer;margin-bottom:30px}div.vis-configuration.vis-config-button.hover{background-color:#4588e6;border:2px solid #214373;color:#fff}div.vis-configuration.vis-config-item{display:block;float:left;width:495px;height:25px;vertical-align:middle;line-height:25px}div.vis-configuration.vis-config-item.vis-config-s2{left:10px;background-color:#f7f8fa;padding-left:5px;border-radius:3px}div.vis-configuration.vis-config-item.vis-config-s3{left:20px;background-color:#e4e9f0;padding-left:5px;border-radius:3px}div.vis-configuration.vis-config-item.vis-config-s4{left:30px;background-color:#cfd8e6;padding-left:5px;border-radius:3px}div.vis-configuration.vis-config-header{font-size:18px;font-weight:700}div.vis-configuration.vis-config-label{width:120px;height:25px;line-height:25px}div.vis-configuration.vis-config-label.vis-config-s3{width:110px}div.vis-configuration.vis-config-label.vis-config-s4{width:100px}div.vis-configuration.vis-config-colorBlock{top:1px;width:30px;height:19px;border:1px solid #444;border-radius:2px;padding:0;margin:0;cursor:pointer}input.vis-configuration.vis-config-checkbox{left:-5px}input.vis-configuration.vis-config-rangeinput{position:relative;top:-5px;width:60px;padding:1px;margin:0;pointer-events:none}.vis-panel,.vis-timeline{padding:0;box-sizing:border-box}input.vis-configuration.vis-config-range{-webkit-appearance:none;border:0 solid #fff;background-color:rgba(0,0,0,0);width:300px;height:20px}input.vis-configuration.vis-config-range::-webkit-slider-runnable-track{width:300px;height:5px;background:#dedede;background:-moz-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#dedede),color-stop(99%,#c8c8c8));background:-webkit-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-o-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-ms-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:linear-gradient(to bottom,#dedede 0,#c8c8c8 99%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#dedede', endColorstr='#c8c8c8', GradientType=0 );border:1px solid #999;box-shadow:#aaa 0 0 3px 0;border-radius:3px}input.vis-configuration.vis-config-range::-webkit-slider-thumb{-webkit-appearance:none;border:1px solid #14334b;height:17px;width:17px;border-radius:50%;background:#3876c2;background:-moz-linear-gradient(top,#3876c2 0,#385380 100%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#3876c2),color-stop(100%,#385380));background:-webkit-linear-gradient(top,#3876c2 0,#385380 100%);background:-o-linear-gradient(top,#3876c2 0,#385380 100%);background:-ms-linear-gradient(top,#3876c2 0,#385380 100%);background:linear-gradient(to bottom,#3876c2 0,#385380 100%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#3876c2', endColorstr='#385380', GradientType=0 );box-shadow:#111927 0 0 1px 0;margin-top:-7px}input.vis-configuration.vis-config-range:focus{outline:0}input.vis-configuration.vis-config-range:focus::-webkit-slider-runnable-track{background:#9d9d9d;background:-moz-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#9d9d9d),color-stop(99%,#c8c8c8));background:-webkit-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:-o-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:-ms-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:linear-gradient(to bottom,#9d9d9d 0,#c8c8c8 99%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#9d9d9d', endColorstr='#c8c8c8', GradientType=0 )}input.vis-configuration.vis-config-range::-moz-range-track{width:300px;height:10px;background:#dedede;background:-moz-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#dedede),color-stop(99%,#c8c8c8));background:-webkit-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-o-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-ms-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:linear-gradient(to bottom,#dedede 0,#c8c8c8 99%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#dedede', endColorstr='#c8c8c8', GradientType=0 );border:1px solid #999;box-shadow:#aaa 0 0 3px 0;border-radius:3px}input.vis-configuration.vis-config-range::-moz-range-thumb{border:none;height:16px;width:16px;border-radius:50%;background:#385380}input.vis-configuration.vis-config-range:-moz-focusring{outline:#fff solid 1px;outline-offset:-1px}input.vis-configuration.vis-config-range::-ms-track{width:300px;height:5px;background:0 0;border-color:transparent;border-width:6px 0;color:transparent}input.vis-configuration.vis-config-range::-ms-fill-lower{background:#777;border-radius:10px}input.vis-configuration.vis-config-range::-ms-fill-upper{background:#ddd;border-radius:10px}input.vis-configuration.vis-config-range::-ms-thumb{border:none;height:16px;width:16px;border-radius:50%;background:#385380}input.vis-configuration.vis-config-range:focus::-ms-fill-lower{background:#888}input.vis-configuration.vis-config-range:focus::-ms-fill-upper{background:#ccc}.vis-configuration-popup{position:absolute;background:rgba(57,76,89,.85);border:2px solid #f2faff;line-height:30px;height:30px;width:150px;text-align:center;color:#fff;font-size:14px;border-radius:4px;-webkit-transition:opacity .3s ease-in-out;-moz-transition:opacity .3s ease-in-out;transition:opacity .3s ease-in-out}.vis-configuration-popup:after,.vis-configuration-popup:before{left:100%;top:50%;border:solid transparent;content:" ";height:0;width:0;position:absolute;pointer-events:none}.vis-configuration-popup:after{border-color:rgba(136,183,213,0);border-left-color:rgba(57,76,89,.85);border-width:8px;margin-top:-8px}.vis-configuration-popup:before{border-color:rgba(194,225,245,0);border-left-color:#f2faff;border-width:12px;margin-top:-12px}.vis-timeline{position:relative;border:1px solid #bfbfbf;margin:0}.vis-panel{position:absolute;margin:0}.vis-panel.vis-bottom,.vis-panel.vis-center,.vis-panel.vis-left,.vis-panel.vis-right,.vis-panel.vis-top{border:1px #bfbfbf}.vis-panel.vis-center,.vis-panel.vis-left,.vis-panel.vis-right{border-top-style:solid;border-bottom-style:solid;overflow:hidden}.vis-panel.vis-bottom,.vis-panel.vis-center,.vis-panel.vis-top{border-left-style:solid;border-right-style:solid}.vis-panel>.vis-content{position:relative}.vis-panel .vis-shadow{position:absolute;width:100%;height:1px;box-shadow:0 0 10px rgba(0,0,0,.8)}.vis-itemset,.vis-labelset,.vis-labelset .vis-label{position:relative;box-sizing:border-box}.vis-panel .vis-shadow.vis-top{top:-1px;left:0}.vis-panel .vis-shadow.vis-bottom{bottom:-1px;left:0}.vis-labelset .vis-label{left:0;top:0;width:100%;color:#4d4d4d;border-bottom:1px solid #bfbfbf}.vis-labelset .vis-label.draggable{cursor:pointer}.vis-labelset .vis-label:last-child{border-bottom:none}.vis-labelset .vis-label .vis-inner{display:inline-block;padding:5px}.vis-labelset .vis-label .vis-inner.vis-hidden{padding:0}.vis-itemset{padding:0;margin:0}.vis-itemset .vis-background,.vis-itemset .vis-foreground{position:absolute;width:100%;height:100%;overflow:visible}.vis-axis{position:absolute;width:100%;height:0;left:0;z-index:1}.vis-foreground .vis-group{position:relative;box-sizing:border-box;border-bottom:1px solid #bfbfbf}.vis-foreground .vis-group:last-child{border-bottom:none}.vis-overlay{position:absolute;top:0;left:0;width:100%;height:100%;z-index:10}.vis-item{position:absolute;color:#1A1A1A;border-color:#97B0F8;border-width:1px;background-color:#D5DDF6;display:inline-block}.vis-item.vis-point.vis-selected,.vis-item.vis-selected{background-color:#FFF785}.vis-item.vis-selected{border-color:#FFC200;z-index:2}.vis-editable.vis-selected{cursor:move}.vis-item.vis-box{text-align:center;border-style:solid;border-radius:2px}.vis-item.vis-point{background:0 0}.vis-item.vis-dot{position:absolute;padding:0;border-width:4px;border-style:solid;border-radius:4px}.vis-item.vis-range{border-style:solid;border-radius:2px;box-sizing:border-box}.vis-item.vis-background{border:none;background-color:rgba(213,221,246,.4);box-sizing:border-box;padding:0;margin:0}.vis-item .vis-item-overflow{position:relative;width:100%;height:100%;padding:0;margin:0;overflow:hidden}.vis-item .vis-delete,.vis-item .vis-delete-rtl{background:url(img/timeline/delete.png) center no-repeat;height:24px;top:-4px;cursor:pointer}.vis-item.vis-range .vis-item-content{position:relative;display:inline-block}.vis-item.vis-background .vis-item-content{position:absolute;display:inline-block}.vis-item.vis-line{padding:0;position:absolute;width:0;border-left-width:1px;border-left-style:solid}.vis-item .vis-item-content{white-space:nowrap;box-sizing:border-box;padding:5px}.vis-item .vis-delete{position:absolute;width:24px;right:-24px}.vis-item .vis-delete-rtl{position:absolute;width:24px;left:-24px}.vis-item.vis-range .vis-drag-left{position:absolute;width:24px;max-width:20%;min-width:2px;height:100%;top:0;left:-4px;cursor:w-resize}.vis-item.vis-range .vis-drag-right{position:absolute;width:24px;max-width:20%;min-width:2px;height:100%;top:0;right:-4px;cursor:e-resize}.vis-range.vis-item.vis-readonly .vis-drag-left,.vis-range.vis-item.vis-readonly .vis-drag-right{cursor:auto}.vis-time-axis{position:relative;overflow:hidden}.vis-time-axis.vis-foreground{top:0;left:0;width:100%}.vis-time-axis.vis-background{position:absolute;top:0;left:0;width:100%;height:100%}.vis-time-axis .vis-text{position:absolute;color:#4d4d4d;padding:3px;overflow:hidden;box-sizing:border-box;white-space:nowrap}.vis-time-axis .vis-text.vis-measure{position:absolute;padding-left:0;padding-right:0;margin-left:0;margin-right:0;visibility:hidden}.vis-time-axis .vis-grid.vis-vertical{position:absolute;border-left:1px solid}.vis-time-axis .vis-grid.vis-vertical-rtl{position:absolute;border-right:1px solid}.vis-time-axis .vis-grid.vis-minor{border-color:#e5e5e5}.vis-time-axis .vis-grid.vis-major{border-color:#bfbfbf}.vis-current-time{background-color:#FF7F6E;width:2px;z-index:1}.vis-custom-time{background-color:#6E94FF;width:2px;cursor:move;z-index:1}div.vis-network div.vis-close,div.vis-network div.vis-edit-mode div.vis-button,div.vis-network div.vis-manipulation div.vis-button{cursor:pointer;-webkit-user-select:none;-moz-user-select:none;-ms-user-select:none;-webkit-touch-callout:none;-khtml-user-select:none}.vis-panel.vis-background.vis-horizontal .vis-grid.vis-horizontal{position:absolute;width:100%;height:0;border-bottom:1px solid}.vis-panel.vis-background.vis-horizontal .vis-grid.vis-minor{border-color:#e5e5e5}.vis-panel.vis-background.vis-horizontal .vis-grid.vis-major{border-color:#bfbfbf}.vis-data-axis .vis-y-axis.vis-major{width:100%;position:absolute;color:#4d4d4d;white-space:nowrap}.vis-data-axis .vis-y-axis.vis-major.vis-measure{padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis-data-axis .vis-y-axis.vis-minor{position:absolute;width:100%;color:#bebebe;white-space:nowrap}.vis-data-axis .vis-y-axis.vis-minor.vis-measure{padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis-data-axis .vis-y-axis.vis-title{position:absolute;color:#4d4d4d;white-space:nowrap;bottom:20px;text-align:center}.vis-data-axis .vis-y-axis.vis-title.vis-measure{padding:0;margin:0;visibility:hidden;width:auto}.vis-data-axis .vis-y-axis.vis-title.vis-left{bottom:0;-webkit-transform-origin:left top;-moz-transform-origin:left top;-ms-transform-origin:left top;-o-transform-origin:left top;transform-origin:left bottom;-webkit-transform:rotate(-90deg);-moz-transform:rotate(-90deg);-ms-transform:rotate(-90deg);-o-transform:rotate(-90deg);transform:rotate(-90deg)}.vis-data-axis .vis-y-axis.vis-title.vis-right{bottom:0;-webkit-transform-origin:right bottom;-moz-transform-origin:right bottom;-ms-transform-origin:right bottom;-o-transform-origin:right bottom;transform-origin:right bottom;-webkit-transform:rotate(90deg);-moz-transform:rotate(90deg);-ms-transform:rotate(90deg);-o-transform:rotate(90deg);transform:rotate(90deg)}.vis-legend{background-color:rgba(247,252,255,.65);padding:5px;border:1px solid #b3b3b3;box-shadow:2px 2px 10px rgba(154,154,154,.55)}.vis-legend-text{white-space:nowrap;display:inline-block}.vis-graph-group0{fill:#4f81bd;fill-opacity:0;stroke-width:2px;stroke:#4f81bd}.vis-graph-group1{fill:#f79646;fill-opacity:0;stroke-width:2px;stroke:#f79646}.vis-graph-group2{fill:#8c51cf;fill-opacity:0;stroke-width:2px;stroke:#8c51cf}.vis-graph-group3{fill:#75c841;fill-opacity:0;stroke-width:2px;stroke:#75c841}.vis-graph-group4{fill:#ff0100;fill-opacity:0;stroke-width:2px;stroke:#ff0100}.vis-graph-group5{fill:#37d8e6;fill-opacity:0;stroke-width:2px;stroke:#37d8e6}.vis-graph-group6{fill:#042662;fill-opacity:0;stroke-width:2px;stroke:#042662}.vis-graph-group7{fill:#00ff26;fill-opacity:0;stroke-width:2px;stroke:#00ff26}.vis-graph-group8{fill:#f0f;fill-opacity:0;stroke-width:2px;stroke:#f0f}.vis-graph-group9{fill:#8f3938;fill-opacity:0;stroke-width:2px;stroke:#8f3938}.vis-timeline .vis-fill{fill-opacity:.1;stroke:none}.vis-timeline .vis-bar{fill-opacity:.5;stroke-width:1px}.vis-timeline .vis-point{stroke-width:2px;fill-opacity:1}.vis-timeline .vis-legend-background{stroke-width:1px;fill-opacity:.9;fill:#fff;stroke:#c2c2c2}.vis-timeline .vis-outline{stroke-width:1px;fill-opacity:1;fill:#fff;stroke:#e5e5e5}.vis-timeline .vis-icon-fill{fill-opacity:.3;stroke:none}div.vis-network div.vis-manipulation{border-width:0;border-bottom:1px;border-style:solid;border-color:#d6d9d8;background:#fff;background:-moz-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#fff),color-stop(48%,#fcfcfc),color-stop(50%,#fafafa),color-stop(100%,#fcfcfc));background:-webkit-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-o-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-ms-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:linear-gradient(to bottom,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#ffffff', endColorstr='#fcfcfc', GradientType=0 );padding-top:4px;position:absolute;left:0;top:0;width:100%;height:28px}div.vis-network div.vis-edit-mode{position:absolute;left:0;top:5px;height:30px}div.vis-network div.vis-close{position:absolute;right:0;top:0;width:30px;height:30px;background-position:20px 3px;background-repeat:no-repeat;background-image:url(img/network/cross.png);user-select:none}div.vis-network div.vis-close:hover{opacity:.6}div.vis-network div.vis-edit-mode div.vis-button,div.vis-network div.vis-manipulation div.vis-button{float:left;font-family:verdana;font-size:12px;-moz-border-radius:15px;border-radius:15px;display:inline-block;background-position:0 0;background-repeat:no-repeat;height:24px;margin-left:10px;padding:0 8px;user-select:none}div.vis-network div.vis-manipulation div.vis-button:hover{box-shadow:1px 1px 8px rgba(0,0,0,.2)}div.vis-network div.vis-manipulation div.vis-button:active{box-shadow:1px 1px 8px rgba(0,0,0,.5)}div.vis-network div.vis-manipulation div.vis-button.vis-back{background-image:url(img/network/backIcon.png)}div.vis-network div.vis-manipulation div.vis-button.vis-none:hover{box-shadow:1px 1px 8px transparent;cursor:default}div.vis-network div.vis-manipulation div.vis-button.vis-none:active{box-shadow:1px 1px 8px transparent}div.vis-network div.vis-manipulation div.vis-button.vis-none{padding:0}div.vis-network div.vis-manipulation div.notification{margin:2px;font-weight:700}div.vis-network div.vis-manipulation div.vis-button.vis-add{background-image:url(img/network/addNodeIcon.png)}div.vis-network div.vis-edit-mode div.vis-button.vis-edit,div.vis-network div.vis-manipulation div.vis-button.vis-edit{background-image:url(img/network/editIcon.png)}div.vis-network div.vis-edit-mode div.vis-button.vis-edit.vis-edit-mode{background-color:#fcfcfc;border:1px solid #ccc}div.vis-network div.vis-manipulation div.vis-button.vis-connect{background-image:url(img/network/connectIcon.png)}div.vis-network div.vis-manipulation div.vis-button.vis-delete{background-image:url(img/network/deleteIcon.png)}div.vis-network div.vis-edit-mode div.vis-label,div.vis-network div.vis-manipulation div.vis-label{margin:0 0 0 23px;line-height:25px}div.vis-network div.vis-manipulation div.vis-separator-line{float:left;display:inline-block;width:1px;height:21px;background-color:#bdbdbd;margin:0 7px 0 15px}div.vis-network-tooltip{position:absolute;visibility:hidden;padding:5px;white-space:nowrap;font-family:verdana;font-size:14px;color:#000;background-color:#f5f4ed;-moz-border-radius:3px;-webkit-border-radius:3px;border-radius:3px;border:1px solid #808074;box-shadow:3px 3px 10px rgba(0,0,0,.2);pointer-events:none}div.vis-network div.vis-navigation div.vis-button{width:34px;height:34px;-moz-border-radius:17px;border-radius:17px;position:absolute;display:inline-block;background-position:2px 2px;background-repeat:no-repeat;cursor:pointer;-webkit-touch-callout:none;-webkit-user-select:none;-khtml-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}div.vis-network div.vis-navigation div.vis-button:hover{box-shadow:0 0 3px 3px rgba(56,207,21,.3)}div.vis-network div.vis-navigation div.vis-button:active{box-shadow:0 0 1px 3px rgba(56,207,21,.95)}div.vis-network div.vis-navigation div.vis-button.vis-up{background-image:url(img/network/upArrow.png);bottom:50px;left:55px}div.vis-network div.vis-navigation div.vis-button.vis-down{background-image:url(img/network/downArrow.png);bottom:10px;left:55px}div.vis-network div.vis-navigation div.vis-button.vis-left{background-image:url(img/network/leftArrow.png);bottom:10px;left:15px}div.vis-network div.vis-navigation div.vis-button.vis-right{background-image:url(img/network/rightArrow.png);bottom:10px;left:95px}div.vis-network div.vis-navigation div.vis-button.vis-zoomIn{background-image:url(img/network/plus.png);bottom:10px;right:15px}div.vis-network div.vis-navigation div.vis-button.vis-zoomOut{background-image:url(img/network/minus.png);bottom:10px;right:55px}div.vis-network div.vis-navigation div.vis-button.vis-zoomExtends{background-image:url(img/network/zoomExtends.png);bottom:50px;right:15px}div.vis-color-picker{position:absolute;top:0;left:30px;margin-top:-140px;margin-left:30px;width:310px;height:444px;z-index:1;padding:10px;border-radius:15px;background-color:#fff;display:none;box-shadow:rgba(0,0,0,.5) 0 0 10px 0}div.vis-color-picker div.vis-arrow{position:absolute;top:147px;left:5px}div.vis-color-picker div.vis-arrow::after,div.vis-color-picker div.vis-arrow::before{right:100%;top:50%;border:solid transparent;content:" ";height:0;width:0;position:absolute;pointer-events:none}div.vis-color-picker div.vis-arrow:after{border-color:rgba(255,255,255,0);border-right-color:#fff;border-width:30px;margin-top:-30px}div.vis-color-picker div.vis-color{position:absolute;width:289px;height:289px;cursor:pointer}div.vis-color-picker div.vis-brightness{position:absolute;top:313px}div.vis-color-picker div.vis-opacity{position:absolute;top:350px}div.vis-color-picker div.vis-selector{position:absolute;top:137px;left:137px;width:15px;height:15px;border-radius:15px;border:1px solid #fff;background:#4c4c4c;background:-moz-linear-gradient(top,#4c4c4c 0,#595959 12%,#666 25%,#474747 39%,#2c2c2c 50%,#000 51%,#111 60%,#2b2b2b 76%,#1c1c1c 91%,#131313 100%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#4c4c4c),color-stop(12%,#595959),color-stop(25%,#666),color-stop(39%,#474747),color-stop(50%,#2c2c2c),color-stop(51%,#000),color-stop(60%,#111),color-stop(76%,#2b2b2b),color-stop(91%,#1c1c1c),color-stop(100%,#131313));background:-webkit-linear-gradient(top,#4c4c4c 0,#595959 12%,#666 25%,#474747 39%,#2c2c2c 50%,#000 51%,#111 60%,#2b2b2b 76%,#1c1c1c 91%,#131313 100%);background:-o-linear-gradient(top,#4c4c4c 0,#595959 12%,#666 25%,#474747 39%,#2c2c2c 50%,#000 51%,#111 60%,#2b2b2b 76%,#1c1c1c 91%,#131313 100%);background:-ms-linear-gradient(top,#4c4c4c 0,#595959 12%,#666 25%,#474747 39%,#2c2c2c 50%,#000 51%,#111 60%,#2b2b2b 76%,#1c1c1c 91%,#131313 100%);background:linear-gradient(to bottom,#4c4c4c 0,#595959 12%,#666 25%,#474747 39%,#2c2c2c 50%,#000 51%,#111 60%,#2b2b2b 76%,#1c1c1c 91%,#131313 100%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#4c4c4c', endColorstr='#131313', GradientType=0 )}div.vis-color-picker div.vis-initial-color,div.vis-color-picker div.vis-new-color{width:140px;height:20px;top:380px;font-size:10px;color:rgba(0,0,0,.4);line-height:20px;position:absolute;vertical-align:middle}div.vis-color-picker div.vis-new-color{border:1px solid rgba(0,0,0,.1);border-radius:5px;left:159px;text-align:right;padding-right:2px}div.vis-color-picker div.vis-initial-color{border:1px solid rgba(0,0,0,.1);border-radius:5px;left:10px;text-align:left;padding-left:2px}div.vis-color-picker div.vis-label{position:absolute;width:300px;left:10px}div.vis-color-picker div.vis-label.vis-brightness{top:300px}div.vis-color-picker div.vis-label.vis-opacity{top:338px}div.vis-color-picker div.vis-button{position:absolute;width:68px;height:25px;border-radius:10px;vertical-align:middle;text-align:center;line-height:25px;top:410px;border:2px solid #d9d9d9;background-color:#f7f7f7;cursor:pointer}div.vis-color-picker div.vis-button.vis-cancel{left:5px}div.vis-color-picker div.vis-button.vis-load{left:82px}div.vis-color-picker div.vis-button.vis-apply{left:159px}div.vis-color-picker div.vis-button.vis-save{left:236px}div.vis-color-picker input.vis-range{width:290px;height:20px} \ No newline at end of file diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis.min.js b/core/src/main/resources/org/apache/spark/ui/static/vis.min.js index 2b3b1d60463f..92b8ed75d85f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/vis.min.js +++ b/core/src/main/resources/org/apache/spark/ui/static/vis.min.js @@ -4,11 +4,11 @@ * * A dynamic, browser-based visualization library. * - * @version 3.9.0 - * @date 2015-01-16 + * @version 4.16.1 + * @date 2016-04-18 * * @license - * Copyright (C) 2011-2014 Almende B.V, http://almende.com + * Copyright (C) 2011-2016 Almende B.V, http://almende.com * * Vis.js is dual licensed under both * @@ -22,17 +22,24 @@ * * Vis.js may be distributed under either license. */ -"use strict";!function(t,e){"object"==typeof exports&&"object"==typeof module?module.exports=e():"function"==typeof define&&define.amd?define(e):"object"==typeof exports?exports.vis=e():t.vis=e()}(this,function(){return function(t){function e(s){if(i[s])return i[s].exports;var o=i[s]={exports:{},id:s,loaded:!1};return t[s].call(o.exports,o,o.exports,e),o.loaded=!0,o.exports}var i={};return e.m=t,e.c=i,e.p="",e(0)}([function(t,e,i){e.util=i(1),e.DOMutil=i(2),e.DataSet=i(3),e.DataView=i(4),e.Queue=i(5),e.Graph3d=i(6),e.graph3d={Camera:i(7),Filter:i(8),Point2d:i(9),Point3d:i(10),Slider:i(11),StepNumber:i(12)},e.Timeline=i(13),e.Graph2d=i(14),e.timeline={DateUtil:i(15),DataStep:i(16),Range:i(17),stack:i(18),TimeStep:i(19),components:{items:{Item:i(31),BackgroundItem:i(32),BoxItem:i(33),PointItem:i(34),RangeItem:i(35)},Component:i(20),CurrentTime:i(21),CustomTime:i(22),DataAxis:i(23),GraphGroup:i(24),Group:i(25),BackgroundGroup:i(26),ItemSet:i(27),Legend:i(28),LineGraph:i(29),TimeAxis:i(30)}},e.Network=i(36),e.network={Edge:i(37),Groups:i(38),Images:i(39),Node:i(40),Popup:i(41),dotparser:i(42),gephiParser:i(43)},e.Graph=function(){throw new Error("Graph is renamed to Network. Please create a graph as new vis.Network(...)")},e.moment=i(44),e.hammer=i(45),e.Hammer=i(45)},function(t,e,i){var s=i(44);e.isNumber=function(t){return t instanceof Number||"number"==typeof t},e.isString=function(t){return t instanceof String||"string"==typeof t},e.isDate=function(t){if(t instanceof Date)return!0;if(e.isString(t)){var i=o.exec(t);if(i)return!0;if(!isNaN(Date.parse(t)))return!0}return!1},e.isDataTable=function(t){return"undefined"!=typeof google&&google.visualization&&google.visualization.DataTable&&t instanceof google.visualization.DataTable},e.randomUUID=function(){var t=function(){return Math.floor(65536*Math.random()).toString(16)};return t()+t()+"-"+t()+"-"+t()+"-"+t()+"-"+t()+t()+t()},e.extend=function(t){for(var e=1,i=arguments.length;i>e;e++){var s=arguments[e];for(var o in s)s.hasOwnProperty(o)&&(t[o]=s[o])}return t},e.selectiveExtend=function(t,e){if(!Array.isArray(t))throw new Error("Array with property names expected as first argument");for(var i=2;ii;i++)if(t[i]!=e[i])return!1;return!0},e.convert=function(t,i){var n;if(void 0===t)return void 0;if(null===t)return null;if(!i)return t;if("string"!=typeof i&&!(i instanceof String))throw new Error("Type must be a string");switch(i){case"boolean":case"Boolean":return Boolean(t);case"number":case"Number":return Number(t.valueOf());case"string":case"String":return String(t);case"Date":if(e.isNumber(t))return new Date(t);if(t instanceof Date)return new Date(t.valueOf());if(s.isMoment(t))return new Date(t.valueOf());if(e.isString(t))return n=o.exec(t),n?new Date(Number(n[1])):s(t).toDate();throw new Error("Cannot convert object of type "+e.getType(t)+" to type Date");case"Moment":if(e.isNumber(t))return s(t);if(t instanceof Date)return s(t.valueOf());if(s.isMoment(t))return s(t);if(e.isString(t))return n=o.exec(t),s(n?Number(n[1]):t);throw new Error("Cannot convert object of type "+e.getType(t)+" to type Date");case"ISODate":if(e.isNumber(t))return new Date(t);if(t instanceof Date)return t.toISOString();if(s.isMoment(t))return t.toDate().toISOString();if(e.isString(t))return n=o.exec(t),n?new Date(Number(n[1])).toISOString():new Date(t).toISOString();throw new Error("Cannot convert object of type "+e.getType(t)+" to type ISODate");case"ASPDate":if(e.isNumber(t))return"/Date("+t+")/";if(t instanceof Date)return"/Date("+t.valueOf()+")/";if(e.isString(t)){n=o.exec(t);var r;return r=n?new Date(Number(n[1])).valueOf():new Date(t).valueOf(),"/Date("+r+")/"}throw new Error("Cannot convert object of type "+e.getType(t)+" to type ASPDate");default:throw new Error('Unknown type "'+i+'"')}};var o=/^\/?Date\((\-?\d+)/i;e.getType=function(t){var e=typeof t;return"object"==e?null==t?"null":t instanceof Boolean?"Boolean":t instanceof Number?"Number":t instanceof String?"String":Array.isArray(t)?"Array":t instanceof Date?"Date":"Object":"number"==e?"Number":"boolean"==e?"Boolean":"string"==e?"String":e},e.getAbsoluteLeft=function(t){return t.getBoundingClientRect().left},e.getAbsoluteTop=function(t){return t.getBoundingClientRect().top},e.addClassName=function(t,e){var i=t.className.split(" ");-1==i.indexOf(e)&&(i.push(e),t.className=i.join(" "))},e.removeClassName=function(t,e){var i=t.className.split(" "),s=i.indexOf(e);-1!=s&&(i.splice(s,1),t.className=i.join(" "))},e.forEach=function(t,e){var i,s;if(Array.isArray(t))for(i=0,s=t.length;s>i;i++)e(t[i],i,t);else for(i in t)t.hasOwnProperty(i)&&e(t[i],i,t)},e.toArray=function(t){var e=[];for(var i in t)t.hasOwnProperty(i)&&e.push(t[i]);return e},e.updateProperty=function(t,e,i){return t[e]!==i?(t[e]=i,!0):!1},e.addEventListener=function(t,e,i,s){t.addEventListener?(void 0===s&&(s=!1),"mousewheel"===e&&navigator.userAgent.indexOf("Firefox")>=0&&(e="DOMMouseScroll"),t.addEventListener(e,i,s)):t.attachEvent("on"+e,i)},e.removeEventListener=function(t,e,i,s){t.removeEventListener?(void 0===s&&(s=!1),"mousewheel"===e&&navigator.userAgent.indexOf("Firefox")>=0&&(e="DOMMouseScroll"),t.removeEventListener(e,i,s)):t.detachEvent("on"+e,i)},e.preventDefault=function(t){t||(t=window.event),t.preventDefault?t.preventDefault():t.returnValue=!1},e.getTarget=function(t){t||(t=window.event);var e;return t.target?e=t.target:t.srcElement&&(e=t.srcElement),void 0!=e.nodeType&&3==e.nodeType&&(e=e.parentNode),e},e.option={},e.option.asBoolean=function(t,e){return"function"==typeof t&&(t=t()),null!=t?0!=t:e||null},e.option.asNumber=function(t,e){return"function"==typeof t&&(t=t()),null!=t?Number(t)||e||null:e||null},e.option.asString=function(t,e){return"function"==typeof t&&(t=t()),null!=t?String(t):e||null},e.option.asSize=function(t,i){return"function"==typeof t&&(t=t()),e.isString(t)?t:e.isNumber(t)?t+"px":i||null},e.option.asElement=function(t,e){return"function"==typeof t&&(t=t()),t||e||null},e.hexToRGB=function(t){var e=/^#?([a-f\d])([a-f\d])([a-f\d])$/i;t=t.replace(e,function(t,e,i,s){return e+e+i+i+s+s});var i=/^#?([a-f\d]{2})([a-f\d]{2})([a-f\d]{2})$/i.exec(t);return i?{r:parseInt(i[1],16),g:parseInt(i[2],16),b:parseInt(i[3],16)}:null},e.RGBToHex=function(t,e,i){return"#"+((1<<24)+(t<<16)+(e<<8)+i).toString(16).slice(1)},e.parseColor=function(t){var i;if(e.isString(t)){if(e.isValidRGB(t)){var s=t.substr(4).substr(0,t.length-5).split(",");t=e.RGBToHex(s[0],s[1],s[2])}if(e.isValidHex(t)){var o=e.hexToHSV(t),n={h:o.h,s:.45*o.s,v:Math.min(1,1.05*o.v)},r={h:o.h,s:Math.min(1,1.25*o.v),v:.6*o.v},a=e.HSVToHex(r.h,r.h,r.v),h=e.HSVToHex(n.h,n.s,n.v);i={background:t,border:a,highlight:{background:h,border:a},hover:{background:h,border:a}}}else i={background:t,border:t,highlight:{background:t,border:t},hover:{background:t,border:t}}}else i={},i.background=t.background||"white",i.border=t.border||i.background,e.isString(t.highlight)?i.highlight={border:t.highlight,background:t.highlight}:(i.highlight={},i.highlight.background=t.highlight&&t.highlight.background||i.background,i.highlight.border=t.highlight&&t.highlight.border||i.border),e.isString(t.hover)?i.hover={border:t.hover,background:t.hover}:(i.hover={},i.hover.background=t.hover&&t.hover.background||i.background,i.hover.border=t.hover&&t.hover.border||i.border);return i},e.RGBToHSV=function(t,e,i){t/=255,e/=255,i/=255;var s=Math.min(t,Math.min(e,i)),o=Math.max(t,Math.max(e,i));if(s==o)return{h:0,s:0,v:s};var n=t==s?e-i:i==s?t-e:i-t,r=t==s?3:i==s?1:5,a=60*(r-n/(o-s))/360,h=(o-s)/o,d=o;return{h:a,s:h,v:d}};var n={split:function(t){var e={};return t.split(";").forEach(function(t){if(""!=t.trim()){var i=t.split(":"),s=i[0].trim(),o=i[1].trim();e[s]=o}}),e},join:function(t){return Object.keys(t).map(function(e){return e+": "+t[e]}).join("; ")}};e.addCssText=function(t,i){var s=n.split(t.style.cssText),o=n.split(i),r=e.extend(s,o);t.style.cssText=n.join(r)},e.removeCssText=function(t,e){var i=n.split(t.style.cssText),s=n.split(e);for(var o in s)s.hasOwnProperty(o)&&delete i[o];t.style.cssText=n.join(i)},e.HSVToRGB=function(t,e,i){var s,o,n,r=Math.floor(6*t),a=6*t-r,h=i*(1-e),d=i*(1-a*e),l=i*(1-(1-a)*e);switch(r%6){case 0:s=i,o=l,n=h;break;case 1:s=d,o=i,n=h;break;case 2:s=h,o=i,n=l;break;case 3:s=h,o=d,n=i;break;case 4:s=l,o=h,n=i;break;case 5:s=i,o=h,n=d}return{r:Math.floor(255*s),g:Math.floor(255*o),b:Math.floor(255*n)}},e.HSVToHex=function(t,i,s){var o=e.HSVToRGB(t,i,s);return e.RGBToHex(o.r,o.g,o.b)},e.hexToHSV=function(t){var i=e.hexToRGB(t);return e.RGBToHSV(i.r,i.g,i.b)},e.isValidHex=function(t){var e=/(^#[0-9A-F]{6}$)|(^#[0-9A-F]{3}$)/i.test(t);return e},e.isValidRGB=function(t){t=t.replace(" ","");var e=/rgb\((\d{1,3}),(\d{1,3}),(\d{1,3})\)/i.test(t);return e},e.selectiveBridgeObject=function(t,i){if("object"==typeof i){for(var s=Object.create(i),o=0;o=r&&o>n;){var h=Math.floor((r+a)/2),d=t[h],l=void 0===s?d[i]:d[i][s],c=e(l);if(0==c)return h;-1==c?r=h+1:a=h-1,n++}return-1},e.binarySearchValue=function(t,e,i,s){for(var o,n,r,a,h=1e4,d=0,l=0,c=t.length-1;c>=l&&h>d;){if(a=Math.floor(.5*(c+l)),o=t[Math.max(0,a-1)][i],n=t[a][i],r=t[Math.min(t.length-1,a+1)][i],n==e)return a;if(e>o&&n>e)return"before"==s?Math.max(0,a-1):a;if(e>n&&r>e)return"before"==s?a:Math.min(t.length-1,a+1);e>n?l=a+1:c=a-1,d++}return-1},e.easeInOutQuad=function(t,e,i,s){var o=i-e;return t/=s/2,1>t?o/2*t*t+e:(t--,-o/2*(t*(t-2)-1)+e)},e.easingFunctions={linear:function(t){return t},easeInQuad:function(t){return t*t},easeOutQuad:function(t){return t*(2-t)},easeInOutQuad:function(t){return.5>t?2*t*t:-1+(4-2*t)*t},easeInCubic:function(t){return t*t*t},easeOutCubic:function(t){return--t*t*t+1},easeInOutCubic:function(t){return.5>t?4*t*t*t:(t-1)*(2*t-2)*(2*t-2)+1},easeInQuart:function(t){return t*t*t*t},easeOutQuart:function(t){return 1- --t*t*t*t},easeInOutQuart:function(t){return.5>t?8*t*t*t*t:1-8*--t*t*t*t},easeInQuint:function(t){return t*t*t*t*t},easeOutQuint:function(t){return 1+--t*t*t*t*t},easeInOutQuint:function(t){return.5>t?16*t*t*t*t*t:1+16*--t*t*t*t*t}}},function(t,e){e.prepareElements=function(t){for(var e in t)t.hasOwnProperty(e)&&(t[e].redundant=t[e].used,t[e].used=[])},e.cleanupElements=function(t){for(var e in t)if(t.hasOwnProperty(e)&&t[e].redundant){for(var i=0;i0?(s=e[t].redundant[0],e[t].redundant.shift()):(s=document.createElementNS("http://www.w3.org/2000/svg",t),i.appendChild(s)):(s=document.createElementNS("http://www.w3.org/2000/svg",t),e[t]={used:[],redundant:[]},i.appendChild(s)),e[t].used.push(s),s},e.getDOMElement=function(t,e,i,s){var o;return e.hasOwnProperty(t)?e[t].redundant.length>0?(o=e[t].redundant[0],e[t].redundant.shift()):(o=document.createElement(t),void 0!==s?i.insertBefore(o,s):i.appendChild(o)):(o=document.createElement(t),e[t]={used:[],redundant:[]},void 0!==s?i.insertBefore(o,s):i.appendChild(o)),e[t].used.push(o),o},e.drawPoint=function(t,i,s,o,n){var r;return"circle"==s.options.drawPoints.style?(r=e.getSVGElement("circle",o,n),r.setAttributeNS(null,"cx",t),r.setAttributeNS(null,"cy",i),r.setAttributeNS(null,"r",.5*s.options.drawPoints.size)):(r=e.getSVGElement("rect",o,n),r.setAttributeNS(null,"x",t-.5*s.options.drawPoints.size),r.setAttributeNS(null,"y",i-.5*s.options.drawPoints.size),r.setAttributeNS(null,"width",s.options.drawPoints.size),r.setAttributeNS(null,"height",s.options.drawPoints.size)),void 0!==s.options.drawPoints.styles&&r.setAttributeNS(null,"style",s.group.options.drawPoints.styles),r.setAttributeNS(null,"class",s.className+" point"),r},e.drawBar=function(t,i,s,o,n,r,a){if(0!=o){0>o&&(o*=-1,i-=o);var h=e.getSVGElement("rect",r,a);h.setAttributeNS(null,"x",t-.5*s),h.setAttributeNS(null,"y",i),h.setAttributeNS(null,"width",s),h.setAttributeNS(null,"height",o),h.setAttributeNS(null,"class",n)}}},function(t,e,i){function s(t,e){if(!t||Array.isArray(t)||o.isDataTable(t)||(e=t,t=null),this._options=e||{},this._data={},this._fieldId=this._options.fieldId||"id",this._type={},this._options.type)for(var i in this._options.type)if(this._options.type.hasOwnProperty(i)){var s=this._options.type[i];this._type[i]="Date"==s||"ISODate"==s||"ASPDate"==s?"Date":s}if(this._options.convert)throw new Error('Option "convert" is deprecated. Use "type" instead.');this._subscribers={},t&&this.add(t),this.setOptions(e)}var o=i(1),n=i(5);s.prototype.setOptions=function(t){t&&void 0!==t.queue&&(t.queue===!1?this._queue&&(this._queue.destroy(),delete this._queue):(this._queue||(this._queue=n.extend(this,{replace:["add","update","remove"]})),"object"==typeof t.queue&&this._queue.setOptions(t.queue)))},s.prototype.on=function(t,e){var i=this._subscribers[t];i||(i=[],this._subscribers[t]=i),i.push({callback:e})},s.prototype.subscribe=s.prototype.on,s.prototype.off=function(t,e){var i=this._subscribers[t];i&&(this._subscribers[t]=i.filter(function(t){return t.callback!=e}))},s.prototype.unsubscribe=s.prototype.off,s.prototype._trigger=function(t,e,i){if("*"==t)throw new Error("Cannot trigger event *");var s=[];t in this._subscribers&&(s=s.concat(this._subscribers[t])),"*"in this._subscribers&&(s=s.concat(this._subscribers["*"]));for(var o=0;or;r++)i=n._addItem(t[r]),s.push(i);else if(o.isDataTable(t))for(var h=this._getColumnNames(t),d=0,l=t.getNumberOfRows();l>d;d++){for(var c={},p=0,u=h.length;u>p;p++){var m=h[p];c[m]=t.getValue(d,p)}i=n._addItem(c),s.push(i)}else{if(!(t instanceof Object))throw new Error("Unknown dataType");i=n._addItem(t),s.push(i)}return s.length&&this._trigger("add",{items:s},e),s},s.prototype.update=function(t,e){var i=[],s=[],n=[],r=this,a=r._fieldId,h=function(t){var e=t[a];r._data[e]?(e=r._updateItem(t),s.push(e),n.push(t)):(e=r._addItem(t),i.push(e))};if(Array.isArray(t))for(var d=0,l=t.length;l>d;d++)h(t[d]);else if(o.isDataTable(t))for(var c=this._getColumnNames(t),p=0,u=t.getNumberOfRows();u>p;p++){for(var m={},f=0,g=c.length;g>f;f++){var v=c[f];m[v]=t.getValue(p,f)}h(m)}else{if(!(t instanceof Object))throw new Error("Unknown dataType");h(t)}return i.length&&this._trigger("add",{items:i},e),s.length&&this._trigger("update",{items:s,data:n},e),i.concat(s)},s.prototype.get=function(){var t,e,i,s,n=this,r=o.getType(arguments[0]);"String"==r||"Number"==r?(t=arguments[0],i=arguments[1],s=arguments[2]):"Array"==r?(e=arguments[0],i=arguments[1],s=arguments[2]):(i=arguments[0],s=arguments[1]);var a;if(i&&i.returnType){var h=["DataTable","Array","Object"];if(a=-1==h.indexOf(i.returnType)?"Array":i.returnType,s&&a!=o.getType(s))throw new Error('Type of parameter "data" ('+o.getType(s)+") does not correspond with specified options.type ("+i.type+")");if("DataTable"==a&&!o.isDataTable(s))throw new Error('Parameter "data" must be a DataTable when options.type is "DataTable"')}else a=s&&"DataTable"==o.getType(s)?"DataTable":"Array";var d,l,c,p,u=i&&i.type||this._options.type,m=i&&i.filter,f=[];if(void 0!=t)d=n._getItem(t,u),m&&!m(d)&&(d=null);else if(void 0!=e)for(c=0,p=e.length;p>c;c++)d=n._getItem(e[c],u),(!m||m(d))&&f.push(d);else for(l in this._data)this._data.hasOwnProperty(l)&&(d=n._getItem(l,u),(!m||m(d))&&f.push(d));if(i&&i.order&&void 0==t&&this._sort(f,i.order),i&&i.fields){var g=i.fields;if(void 0!=t)d=this._filterFields(d,g);else for(c=0,p=f.length;p>c;c++)f[c]=this._filterFields(f[c],g)}if("DataTable"==a){var v=this._getColumnNames(s);if(void 0!=t)n._appendRow(s,v,d);else for(c=0;cc;c++)s.push(f[c]);return s}return f},s.prototype.getIds=function(t){var e,i,s,o,n,r=this._data,a=t&&t.filter,h=t&&t.order,d=t&&t.type||this._options.type,l=[];if(a)if(h){n=[];for(s in r)r.hasOwnProperty(s)&&(o=this._getItem(s,d),a(o)&&n.push(o));for(this._sort(n,h),e=0,i=n.length;i>e;e++)l[e]=n[e][this._fieldId]}else for(s in r)r.hasOwnProperty(s)&&(o=this._getItem(s,d),a(o)&&l.push(o[this._fieldId]));else if(h){n=[];for(s in r)r.hasOwnProperty(s)&&n.push(r[s]);for(this._sort(n,h),e=0,i=n.length;i>e;e++)l[e]=n[e][this._fieldId]}else for(s in r)r.hasOwnProperty(s)&&(o=r[s],l.push(o[this._fieldId]));return l},s.prototype.getDataSet=function(){return this},s.prototype.forEach=function(t,e){var i,s,o=e&&e.filter,n=e&&e.type||this._options.type,r=this._data;if(e&&e.order)for(var a=this.get(e),h=0,d=a.length;d>h;h++)i=a[h],s=i[this._fieldId],t(i,s);else for(s in r)r.hasOwnProperty(s)&&(i=this._getItem(s,n),(!o||o(i))&&t(i,s))},s.prototype.map=function(t,e){var i,s=e&&e.filter,o=e&&e.type||this._options.type,n=[],r=this._data;for(var a in r)r.hasOwnProperty(a)&&(i=this._getItem(a,o),(!s||s(i))&&n.push(t(i,a)));return e&&e.order&&this._sort(n,e.order),n},s.prototype._filterFields=function(t,e){var i={};for(var s in t)t.hasOwnProperty(s)&&-1!=e.indexOf(s)&&(i[s]=t[s]);return i},s.prototype._sort=function(t,e){if(o.isString(e)){var i=e;t.sort(function(t,e){var s=t[i],o=e[i];return s>o?1:o>s?-1:0})}else{if("function"!=typeof e)throw new TypeError("Order must be a function or a string");t.sort(e)}},s.prototype.remove=function(t,e){var i,s,o,n=[];if(Array.isArray(t))for(i=0,s=t.length;s>i;i++)o=this._remove(t[i]),null!=o&&n.push(o);else o=this._remove(t),null!=o&&n.push(o);return n.length&&this._trigger("remove",{items:n},e),n},s.prototype._remove=function(t){if(o.isNumber(t)||o.isString(t)){if(this._data[t])return delete this._data[t],t}else if(t instanceof Object){var e=t[this._fieldId];if(e&&this._data[e])return delete this._data[e],e}return null},s.prototype.clear=function(t){var e=Object.keys(this._data);return this._data={},this._trigger("remove",{items:e},t),e},s.prototype.max=function(t){var e=this._data,i=null,s=null;for(var o in e)if(e.hasOwnProperty(o)){var n=e[o],r=n[t];null!=r&&(!i||r>s)&&(i=n,s=r)}return i},s.prototype.min=function(t){var e=this._data,i=null,s=null;for(var o in e)if(e.hasOwnProperty(o)){var n=e[o],r=n[t];null!=r&&(!i||s>r)&&(i=n,s=r)}return i},s.prototype.distinct=function(t){var e,i=this._data,s=[],n=this._options.type&&this._options.type[t]||null,r=0;for(var a in i)if(i.hasOwnProperty(a)){var h=i[a],d=h[t],l=!1;for(e=0;r>e;e++)if(s[e]==d){l=!0;break}l||void 0===d||(s[r]=d,r++)}if(n)for(e=0;ei;i++)e[i]=t.getColumnId(i)||t.getColumnLabel(i);return e},s.prototype._appendRow=function(t,e,i){for(var s=t.addRow(),o=0,n=e.length;n>o;o++){var r=e[o];t.setValue(s,o,i[r])}},t.exports=s},function(t,e,i){function s(t,e){this._data=null,this._ids={},this._options=e||{},this._fieldId="id",this._subscribers={};var i=this;this.listener=function(){i._onEvent.apply(i,arguments)},this.setData(t)}var o=i(1),n=i(3);s.prototype.setData=function(t){var e,i,s;if(this._data){this._data.unsubscribe&&this._data.unsubscribe("*",this.listener),e=[];for(var o in this._ids)this._ids.hasOwnProperty(o)&&e.push(o);this._ids={},this._trigger("remove",{items:e})}if(this._data=t,this._data){for(this._fieldId=this._options.fieldId||this._data&&this._data.options&&this._data.options.fieldId||"id",e=this._data.getIds({filter:this._options&&this._options.filter}),i=0,s=e.length;s>i;i++)o=e[i],this._ids[o]=!0;this._trigger("add",{items:e}),this._data.on&&this._data.on("*",this.listener)}},s.prototype.get=function(){var t,e,i,s=this,n=o.getType(arguments[0]);"String"==n||"Number"==n||"Array"==n?(t=arguments[0],e=arguments[1],i=arguments[2]):(e=arguments[0],i=arguments[1]);var r=o.extend({},this._options,e);this._options.filter&&e&&e.filter&&(r.filter=function(t){return s._options.filter(t)&&e.filter(t)});var a=[];return void 0!=t&&a.push(t),a.push(r),a.push(i),this._data&&this._data.get.apply(this._data,a)},s.prototype.getIds=function(t){var e;if(this._data){var i,s=this._options.filter;i=t&&t.filter?s?function(e){return s(e)&&t.filter(e)}:t.filter:s,e=this._data.getIds({filter:i,order:t&&t.order})}else e=[];return e},s.prototype.getDataSet=function(){for(var t=this;t instanceof s;)t=t._data;return t||null},s.prototype._onEvent=function(t,e,i){var s,o,n,r,a=e&&e.items,h=this._data,d=[],l=[],c=[];if(a&&h){switch(t){case"add":for(s=0,o=a.length;o>s;s++)n=a[s],r=this.get(n),r&&(this._ids[n]=!0,d.push(n));break;case"update":for(s=0,o=a.length;o>s;s++)n=a[s],r=this.get(n),r?this._ids[n]?l.push(n):(this._ids[n]=!0,d.push(n)):this._ids[n]&&(delete this._ids[n],c.push(n));break;case"remove":for(s=0,o=a.length;o>s;s++)n=a[s],this._ids[n]&&(delete this._ids[n],c.push(n))}d.length&&this._trigger("add",{items:d},i),l.length&&this._trigger("update",{items:l},i),c.length&&this._trigger("remove",{items:c},i)}},s.prototype.on=n.prototype.on,s.prototype.off=n.prototype.off,s.prototype._trigger=n.prototype._trigger,s.prototype.subscribe=s.prototype.on,s.prototype.unsubscribe=s.prototype.off,t.exports=s},function(t){function e(t){this.delay=null,this.max=1/0,this._queue=[],this._timeout=null,this._extended=null,this.setOptions(t)}e.prototype.setOptions=function(t){t&&"undefined"!=typeof t.delay&&(this.delay=t.delay),t&&"undefined"!=typeof t.max&&(this.max=t.max),this._flushIfNeeded()},e.extend=function(t,i){var s=new e(i);if(void 0!==t.flush)throw new Error("Target object already has a property flush");t.flush=function(){s.flush()};var o=[{name:"flush",original:void 0}];if(i&&i.replace)for(var n=0;nthis.max&&this.flush(),clearTimeout(this._timeout),this.queue.length>0&&"number"==typeof this.delay){var t=this;this._timeout=setTimeout(function(){t.flush()},this.delay)}},e.prototype.flush=function(){for(;this._queue.length>0;){var t=this._queue.shift();t.fn.apply(t.context||t.fn,t.args||[])}},t.exports=e},function(t,e,i){function s(t,e,i){if(!(this instanceof s))throw new SyntaxError("Constructor must be called with the new operator");this.containerElement=t,this.width="400px",this.height="400px",this.margin=10,this.defaultXCenter="55%",this.defaultYCenter="50%",this.xLabel="x",this.yLabel="y",this.zLabel="z";var o=function(t){return t};this.xValueLabel=o,this.yValueLabel=o,this.zValueLabel=o,this.filterLabel="time",this.legendLabel="value",this.style=s.STYLE.DOT,this.showPerspective=!0,this.showGrid=!0,this.keepAspectRatio=!0,this.showShadow=!1,this.showGrayBottom=!1,this.showTooltip=!1,this.verticalRatio=.5,this.animationInterval=1e3,this.animationPreload=!1,this.camera=new p,this.eye=new l(0,0,-1),this.dataTable=null,this.dataPoints=null,this.colX=void 0,this.colY=void 0,this.colZ=void 0,this.colValue=void 0,this.colFilter=void 0,this.xMin=0,this.xStep=void 0,this.xMax=1,this.yMin=0,this.yStep=void 0,this.yMax=1,this.zMin=0,this.zStep=void 0,this.zMax=1,this.valueMin=0,this.valueMax=1,this.xBarWidth=1,this.yBarWidth=1,this.colorAxis="#4D4D4D",this.colorGrid="#D3D3D3",this.colorDot="#7DC1FF",this.colorDotBorder="#3267D2",this.create(),this.setOptions(i),e&&this.setData(e)}function o(t){return"clientX"in t?t.clientX:t.targetTouches[0]&&t.targetTouches[0].clientX||0}function n(t){return"clientY"in t?t.clientY:t.targetTouches[0]&&t.targetTouches[0].clientY||0}var r=i(56),a=i(3),h=i(4),d=i(1),l=i(10),c=i(9),p=i(7),u=i(8),m=i(11),f=i(12);r(s.prototype),s.prototype._setScale=function(){this.scale=new l(1/(this.xMax-this.xMin),1/(this.yMax-this.yMin),1/(this.zMax-this.zMin)),this.keepAspectRatio&&(this.scale.x3&&(this.colFilter=3);else{if(this.style!==s.STYLE.DOTCOLOR&&this.style!==s.STYLE.DOTSIZE&&this.style!==s.STYLE.BARCOLOR&&this.style!==s.STYLE.BARSIZE)throw'Unknown style "'+this.style+'"';this.colX=0,this.colY=1,this.colZ=2,this.colValue=3,t.getNumberOfColumns()>4&&(this.colFilter=4)}},s.prototype.getNumberOfRows=function(t){return t.length},s.prototype.getNumberOfColumns=function(t){var e=0;for(var i in t[0])t[0].hasOwnProperty(i)&&e++;return e},s.prototype.getDistinctValues=function(t,e){for(var i=[],s=0;st[s][e]&&(i.min=t[s][e]),i.maxt;t++){var m=(t-p)/(u-p),g=240*m,v=this._hsv2rgb(g,1,1);c.strokeStyle=v,c.beginPath(),c.moveTo(h,r+t),c.lineTo(a,r+t),c.stroke()}c.strokeStyle=this.colorAxis,c.strokeRect(h,r,i,n)}if(this.style===s.STYLE.DOTSIZE&&(c.strokeStyle=this.colorAxis,c.fillStyle=this.colorDot,c.beginPath(),c.moveTo(h,r),c.lineTo(a,r),c.lineTo(a-i+e,d),c.lineTo(h,d),c.closePath(),c.fill(),c.stroke()),this.style===s.STYLE.DOTCOLOR||this.style===s.STYLE.DOTSIZE){var y=5,b=new f(this.valueMin,this.valueMax,(this.valueMax-this.valueMin)/5,!0);for(b.start(),b.getCurrent()0?this.yMin:this.yMax,o=this._convert3Dto2D(new l(x,r,this.zMin)),Math.cos(2*_)>0?(g.textAlign="center",g.textBaseline="top",o.y+=b):Math.sin(2*_)<0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText(" "+this.xValueLabel(i.getCurrent())+" ",o.x,o.y),i.next()}for(g.lineWidth=1,s=void 0===this.defaultYStep,i=new f(this.yMin,this.yMax,this.yStep,s),i.start(),i.getCurrent()0?this.xMin:this.xMax,o=this._convert3Dto2D(new l(n,i.getCurrent(),this.zMin)),Math.cos(2*_)<0?(g.textAlign="center",g.textBaseline="top",o.y+=b):Math.sin(2*_)>0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText(" "+this.yValueLabel(i.getCurrent())+" ",o.x,o.y),i.next();for(g.lineWidth=1,s=void 0===this.defaultZStep,i=new f(this.zMin,this.zMax,this.zStep,s),i.start(),i.getCurrent()0?this.xMin:this.xMax,r=Math.sin(_)<0?this.yMin:this.yMax;!i.end();)t=this._convert3Dto2D(new l(n,r,i.getCurrent())),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(t.x-b,t.y),g.stroke(),g.textAlign="right",g.textBaseline="middle",g.fillStyle=this.colorAxis,g.fillText(this.zValueLabel(i.getCurrent())+" ",t.x-5,t.y),i.next();g.lineWidth=1,t=this._convert3Dto2D(new l(n,r,this.zMin)),e=this._convert3Dto2D(new l(n,r,this.zMax)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke(),g.lineWidth=1,p=this._convert3Dto2D(new l(this.xMin,this.yMin,this.zMin)),u=this._convert3Dto2D(new l(this.xMax,this.yMin,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(p.x,p.y),g.lineTo(u.x,u.y),g.stroke(),p=this._convert3Dto2D(new l(this.xMin,this.yMax,this.zMin)),u=this._convert3Dto2D(new l(this.xMax,this.yMax,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(p.x,p.y),g.lineTo(u.x,u.y),g.stroke(),g.lineWidth=1,t=this._convert3Dto2D(new l(this.xMin,this.yMin,this.zMin)),e=this._convert3Dto2D(new l(this.xMin,this.yMax,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke(),t=this._convert3Dto2D(new l(this.xMax,this.yMin,this.zMin)),e=this._convert3Dto2D(new l(this.xMax,this.yMax,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke();var w=this.xLabel;w.length>0&&(c=.1/this.scale.y,n=(this.xMin+this.xMax)/2,r=Math.cos(_)>0?this.yMin-c:this.yMax+c,o=this._convert3Dto2D(new l(n,r,this.zMin)),Math.cos(2*_)>0?(g.textAlign="center",g.textBaseline="top"):Math.sin(2*_)<0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText(w,o.x,o.y));var S=this.yLabel;S.length>0&&(d=.1/this.scale.x,n=Math.sin(_)>0?this.xMin-d:this.xMax+d,r=(this.yMin+this.yMax)/2,o=this._convert3Dto2D(new l(n,r,this.zMin)),Math.cos(2*_)<0?(g.textAlign="center",g.textBaseline="top"):Math.sin(2*_)>0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText(S,o.x,o.y));var M=this.zLabel;M.length>0&&(h=30,n=Math.cos(_)>0?this.xMin:this.xMax,r=Math.sin(_)<0?this.yMin:this.yMax,a=(this.zMin+this.zMax)/2,o=this._convert3Dto2D(new l(n,r,a)),g.textAlign="right",g.textBaseline="middle",g.fillStyle=this.colorAxis,g.fillText(M,o.x-h,o.y))},s.prototype._hsv2rgb=function(t,e,i){var s,o,n,r,a,h;switch(r=i*e,a=Math.floor(t/60),h=r*(1-Math.abs(t/60%2-1)),a){case 0:s=r,o=h,n=0;break;case 1:s=h,o=r,n=0;break;case 2:s=0,o=r,n=h;break;case 3:s=0,o=h,n=r;break;case 4:s=h,o=0,n=r;break;case 5:s=r,o=0,n=h;break;default:s=0,o=0,n=0}return"RGB("+parseInt(255*s)+","+parseInt(255*o)+","+parseInt(255*n)+")"},s.prototype._redrawDataGrid=function(){var t,e,i,o,n,r,a,h,d,c,p,u,m,f=this.frame.canvas,g=f.getContext("2d");if(!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(n=0;n0}else r=!0;r?(m=(t.point.z+e.point.z+i.point.z+o.point.z)/4,c=240*(1-(m-this.zMin)*this.scale.z/this.verticalRatio),p=1,this.showShadow?(u=Math.min(1+S.x/M/2,1),a=this._hsv2rgb(c,p,u),h=a):(u=1,a=this._hsv2rgb(c,p,u),h=this.colorAxis)):(a="gray",h=this.colorAxis),d=.5,g.lineWidth=d,g.fillStyle=a,g.strokeStyle=h,g.beginPath(),g.moveTo(t.screen.x,t.screen.y),g.lineTo(e.screen.x,e.screen.y),g.lineTo(o.screen.x,o.screen.y),g.lineTo(i.screen.x,i.screen.y),g.closePath(),g.fill(),g.stroke()}}else for(n=0;np&&(p=0);var u,m,f;this.style===s.STYLE.DOTCOLOR?(u=240*(1-(d.point.value-this.valueMin)*this.scale.value),m=this._hsv2rgb(u,1,1),f=this._hsv2rgb(u,1,.8)):this.style===s.STYLE.DOTSIZE?(m=this.colorDot,f=this.colorDotBorder):(u=240*(1-(d.point.z-this.zMin)*this.scale.z/this.verticalRatio),m=this._hsv2rgb(u,1,1),f=this._hsv2rgb(u,1,.8)),i.lineWidth=1,i.strokeStyle=f,i.fillStyle=m,i.beginPath(),i.arc(d.screen.x,d.screen.y,p,0,2*Math.PI,!0),i.fill(),i.stroke()}}},s.prototype._redrawDataBar=function(){var t,e,i,o,n=this.frame.canvas,r=n.getContext("2d");if(!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(t=0;t0&&(t=this.dataPoints[0],s.lineWidth=1,s.strokeStyle="blue",s.beginPath(),s.moveTo(t.screen.x,t.screen.y)),e=1;e0&&s.stroke()}},s.prototype._onMouseDown=function(t){if(t=t||window.event,this.leftButtonDown&&this._onMouseUp(t),this.leftButtonDown=t.which?1===t.which:1===t.button,this.leftButtonDown||this.touchDown){this.startMouseX=o(t),this.startMouseY=n(t),this.startStart=new Date(this.start),this.startEnd=new Date(this.end),this.startArmRotation=this.camera.getArmRotation(),this.frame.style.cursor="move";var e=this;this.onmousemove=function(t){e._onMouseMove(t)},this.onmouseup=function(t){e._onMouseUp(t)},d.addEventListener(document,"mousemove",e.onmousemove),d.addEventListener(document,"mouseup",e.onmouseup),d.preventDefault(t)}},s.prototype._onMouseMove=function(t){t=t||window.event;var e=parseFloat(o(t))-this.startMouseX,i=parseFloat(n(t))-this.startMouseY,s=this.startArmRotation.horizontal+e/200,r=this.startArmRotation.vertical+i/200,a=4,h=Math.sin(a/360*2*Math.PI);Math.abs(Math.sin(s))0?1:0>t?-1:0}var s=e[0],o=e[1],n=e[2],r=i((o.x-s.x)*(t.y-s.y)-(o.y-s.y)*(t.x-s.x)),a=i((n.x-o.x)*(t.y-o.y)-(n.y-o.y)*(t.x-o.x)),h=i((s.x-n.x)*(t.y-n.y)-(s.y-n.y)*(t.x-n.x));return!(0!=r&&0!=a&&r!=a||0!=a&&0!=h&&a!=h||0!=r&&0!=h&&r!=h)},s.prototype._dataPointFromXY=function(t,e){var i,o=100,n=null,r=null,a=null,h=new c(t,e);if(this.style===s.STYLE.BAR||this.style===s.STYLE.BARCOLOR||this.style===s.STYLE.BARSIZE)for(i=this.dataPoints.length-1;i>=0;i--){n=this.dataPoints[i];var d=n.surfaces;if(d)for(var l=d.length-1;l>=0;l--){var p=d[l],u=p.corners,m=[u[0].screen,u[1].screen,u[2].screen],f=[u[2].screen,u[3].screen,u[0].screen];if(this._insideTriangle(h,m)||this._insideTriangle(h,f))return n}}else for(i=0;ib)&&o>b&&(a=b,r=n)}}return r},s.prototype._showTooltip=function(t){var e,i,s;this.tooltip?(e=this.tooltip.dom.content,i=this.tooltip.dom.line,s=this.tooltip.dom.dot):(e=document.createElement("div"),e.style.position="absolute",e.style.padding="10px",e.style.border="1px solid #4d4d4d",e.style.color="#1a1a1a",e.style.background="rgba(255,255,255,0.7)",e.style.borderRadius="2px",e.style.boxShadow="5px 5px 10px rgba(128,128,128,0.5)",i=document.createElement("div"),i.style.position="absolute",i.style.height="40px",i.style.width="0",i.style.borderLeft="1px solid #4d4d4d",s=document.createElement("div"),s.style.position="absolute",s.style.height="0",s.style.width="0",s.style.border="5px solid #4d4d4d",s.style.borderRadius="5px",this.tooltip={dataPoint:null,dom:{content:e,line:i,dot:s}}),this._hideTooltip(),this.tooltip.dataPoint=t,e.innerHTML="function"==typeof this.showTooltip?this.showTooltip(t.point):"
x:"+t.point.x+"
y:"+t.point.y+"
z:"+t.point.z+"
",e.style.left="0",e.style.top="0",this.frame.appendChild(e),this.frame.appendChild(i),this.frame.appendChild(s);var o=e.offsetWidth,n=e.offsetHeight,r=i.offsetHeight,a=s.offsetWidth,h=s.offsetHeight,d=t.screen.x-o/2;d=Math.min(Math.max(d,10),this.frame.clientWidth-10-o),i.style.left=t.screen.x+"px",i.style.top=t.screen.y-r+"px",e.style.left=d+"px",e.style.top=t.screen.y-r-n+"px",s.style.left=t.screen.x-a/2+"px",s.style.top=t.screen.y-h/2+"px"},s.prototype._hideTooltip=function(){if(this.tooltip){this.tooltip.dataPoint=null;for(var t in this.tooltip.dom)if(this.tooltip.dom.hasOwnProperty(t)){var e=this.tooltip.dom[t];e&&e.parentNode&&e.parentNode.removeChild(e)}}},t.exports=s},function(t,e,i){function s(){this.armLocation=new o,this.armRotation={},this.armRotation.horizontal=0,this.armRotation.vertical=0,this.armLength=1.7,this.cameraLocation=new o,this.cameraRotation=new o(.5*Math.PI,0,0),this.calculateCameraOrientation()}var o=i(10);s.prototype.setArmLocation=function(t,e,i){this.armLocation.x=t,this.armLocation.y=e,this.armLocation.z=i,this.calculateCameraOrientation()},s.prototype.setArmRotation=function(t,e){void 0!==t&&(this.armRotation.horizontal=t),void 0!==e&&(this.armRotation.vertical=e,this.armRotation.vertical<0&&(this.armRotation.vertical=0),this.armRotation.vertical>.5*Math.PI&&(this.armRotation.vertical=.5*Math.PI)),(void 0!==t||void 0!==e)&&this.calculateCameraOrientation()},s.prototype.getArmRotation=function(){var t={};return t.horizontal=this.armRotation.horizontal,t.vertical=this.armRotation.vertical,t},s.prototype.setArmLength=function(t){void 0!==t&&(this.armLength=t,this.armLength<.71&&(this.armLength=.71),this.armLength>5&&(this.armLength=5),this.calculateCameraOrientation())},s.prototype.getArmLength=function(){return this.armLength},s.prototype.getCameraLocation=function(){return this.cameraLocation},s.prototype.getCameraRotation=function(){return this.cameraRotation},s.prototype.calculateCameraOrientation=function(){this.cameraLocation.x=this.armLocation.x-this.armLength*Math.sin(this.armRotation.horizontal)*Math.cos(this.armRotation.vertical),this.cameraLocation.y=this.armLocation.y-this.armLength*Math.cos(this.armRotation.horizontal)*Math.cos(this.armRotation.vertical),this.cameraLocation.z=this.armLocation.z+this.armLength*Math.sin(this.armRotation.vertical),this.cameraRotation.x=Math.PI/2-this.armRotation.vertical,this.cameraRotation.y=0,this.cameraRotation.z=-this.armRotation.horizontal},t.exports=s},function(t,e,i){function s(t,e,i){this.data=t,this.column=e,this.graph=i,this.index=void 0,this.value=void 0,this.values=i.getDistinctValues(t.get(),this.column),this.values.sort(function(t,e){return t>e?1:e>t?-1:0}),this.values.length>0&&this.selectValue(0),this.dataPoints=[],this.loaded=!1,this.onLoadCallback=void 0,i.animationPreload?(this.loaded=!1,this.loadInBackground()):this.loaded=!0}var o=i(4);s.prototype.isLoaded=function(){return this.loaded},s.prototype.getLoadedProgress=function(){for(var t=this.values.length,e=0;this.dataPoints[e];)e++;return Math.round(e/t*100)},s.prototype.getLabel=function(){return this.graph.filterLabel},s.prototype.getColumn=function(){return this.column},s.prototype.getSelectedValue=function(){return void 0===this.index?void 0:this.values[this.index]},s.prototype.getValues=function(){return this.values},s.prototype.getValue=function(t){if(t>=this.values.length)throw"Error: index out of range";return this.values[t]},s.prototype._getDataPoints=function(t){if(void 0===t&&(t=this.index),void 0===t)return[];var e;if(this.dataPoints[t])e=this.dataPoints[t];else{var i={};i.column=this.column,i.value=this.values[t];var s=new o(this.data,{filter:function(t){return t[i.column]==i.value}}).get();e=this.graph._getDataPoints(s),this.dataPoints[t]=e}return e},s.prototype.setOnLoadCallback=function(t){this.onLoadCallback=t},s.prototype.selectValue=function(t){if(t>=this.values.length)throw"Error: index out of range";this.index=t,this.value=this.values[t]},s.prototype.loadInBackground=function(t){void 0===t&&(t=0);var e=this.graph.frame;if(t0&&(t--,this.setIndex(t))},s.prototype.next=function(){var t=this.getIndex();t0?this.setIndex(0):this.index=void 0},s.prototype.setIndex=function(t){if(!(ts&&(s=0),s>this.values.length-1&&(s=this.values.length-1),s},s.prototype.indexToLeft=function(t){var e=parseFloat(this.frame.bar.style.width)-this.frame.slide.clientWidth-10,i=t/(this.values.length-1)*e,s=i+3;return s},s.prototype._onMouseMove=function(t){var e=t.clientX-this.startClientX,i=this.startSlideX+e,s=this.leftToIndex(i);this.setIndex(s),o.preventDefault()},s.prototype._onMouseUp=function(){this.frame.style.cursor="auto",o.removeEventListener(document,"mousemove",this.onmousemove),o.removeEventListener(document,"mouseup",this.onmouseup),o.preventDefault()},t.exports=s},function(t){function e(t,e,i,s){this._start=0,this._end=0,this._step=1,this.prettyStep=!0,this.precision=5,this._current=0,this.setRange(t,e,i,s)}e.prototype.setRange=function(t,e,i,s){this._start=t?t:0,this._end=e?e:0,this.setStep(i,s)},e.prototype.setStep=function(t,i){void 0===t||0>=t||(void 0!==i&&(this.prettyStep=i),this._step=this.prettyStep===!0?e.calculatePrettyStep(t):t)},e.calculatePrettyStep=function(t){var e=function(t){return Math.log(t)/Math.LN10},i=Math.pow(10,Math.round(e(t))),s=2*Math.pow(10,Math.round(e(t/2))),o=5*Math.pow(10,Math.round(e(t/5))),n=i;return Math.abs(s-t)<=Math.abs(n-t)&&(n=s),Math.abs(o-t)<=Math.abs(n-t)&&(n=o),0>=n&&(n=1),n},e.prototype.getCurrent=function(){return parseFloat(this._current.toPrecision(this.precision))},e.prototype.getStep=function(){return this._step},e.prototype.start=function(){this._current=this._start-this._start%this._step},e.prototype.next=function(){this._current+=this._step},e.prototype.end=function(){return this._current>this._end},t.exports=e},function(t,e,i){function s(t,e,i,r){if(!(this instanceof s))throw new SyntaxError("Constructor must be called with the new operator");if(!(Array.isArray(i)||i instanceof n)&&i instanceof Object){var h=r;r=i,i=h}var u=this;this.defaultOptions={start:null,end:null,autoResize:!0,orientation:"bottom",width:null,height:null,maxHeight:null,minHeight:null},this.options=o.deepExtend({},this.defaultOptions),this._create(t),this.components=[],this.body={dom:this.dom,domProps:this.props,emitter:{on:this.on.bind(this),off:this.off.bind(this),emit:this.emit.bind(this)},hiddenDates:[],util:{snap:null,toScreen:u._toScreen.bind(u),toGlobalScreen:u._toGlobalScreen.bind(u),toTime:u._toTime.bind(u),toGlobalTime:u._toGlobalTime.bind(u)}},this.range=new a(this.body),this.components.push(this.range),this.body.range=this.range,this.timeAxis=new d(this.body),this.components.push(this.timeAxis),this.body.util.snap=this.timeAxis.snap.bind(this.timeAxis),this.currentTime=new l(this.body),this.components.push(this.currentTime),this.customTime=new c(this.body),this.components.push(this.customTime),this.itemSet=new p(this.body),this.components.push(this.itemSet),this.itemsData=null,this.groupsData=null,r&&this.setOptions(r),i&&this.setGroups(i),e?this.setItems(e):this.redraw()}var o=(i(56),i(45),i(1)),n=i(3),r=i(4),a=i(17),h=i(46),d=i(30),l=i(21),c=i(22),p=i(27);s.prototype=new h,s.prototype.setItems=function(t){var e,i=null==this.itemsData;if(e=t?t instanceof n||t instanceof r?t:new n(t,{type:{start:"Date",end:"Date"}}):null,this.itemsData=e,this.itemSet&&this.itemSet.setItems(e),i)if(void 0!=this.options.start||void 0!=this.options.end){if(void 0==this.options.start||void 0==this.options.end)var s=this._getDataRange();var o=void 0!=this.options.start?this.options.start:s.start,a=void 0!=this.options.end?this.options.end:s.end;this.setWindow(o,a,{animate:!1})}else this.fit({animate:!1})},s.prototype.setGroups=function(t){var e;e=t?t instanceof n||t instanceof r?t:new n(t):null,this.groupsData=e,this.itemSet.setGroups(e)},s.prototype.setSelection=function(t,e){this.itemSet&&this.itemSet.setSelection(t),e&&e.focus&&this.focus(t,e)},s.prototype.getSelection=function(){return this.itemSet&&this.itemSet.getSelection()||[]},s.prototype.focus=function(t,e){if(this.itemsData&&void 0!=t){var i=Array.isArray(t)?t:[t],s=this.itemsData.getDataSet().get(i,{type:{start:"Date",end:"Date"}}),o=null,n=null;if(s.forEach(function(t){var e=t.start.valueOf(),i="end"in t?t.end.valueOf():t.start.valueOf();(null===o||o>e)&&(o=e),(null===n||i>n)&&(n=i)}),null!==o&&null!==n){var r=(o+n)/2,a=Math.max(this.range.end-this.range.start,1.1*(n-o)),h=e&&void 0!==e.animate?e.animate:!0;this.range.setRange(r-a/2,r+a/2,h)}}},s.prototype.getItemRange=function(){var t=this.itemsData.getDataSet(),e=null,i=null;if(t){var s=t.min("start");e=s?o.convert(s.start,"Date").valueOf():null;var n=t.max("start");n&&(i=o.convert(n.start,"Date").valueOf());var r=t.max("end");r&&(i=null==i?o.convert(r.end,"Date").valueOf():Math.max(i,o.convert(r.end,"Date").valueOf()))}return{min:null!=e?new Date(e):null,max:null!=i?new Date(i):null}},t.exports=s},function(t,e,i){function s(t,e,i,s){if(!(Array.isArray(i)||i instanceof n)&&i instanceof Object){var r=s;s=i,i=r}var h=this;this.defaultOptions={start:null,end:null,autoResize:!0,orientation:"bottom",width:null,height:null,maxHeight:null,minHeight:null},this.options=o.deepExtend({},this.defaultOptions),this._create(t),this.components=[],this.body={dom:this.dom,domProps:this.props,emitter:{on:this.on.bind(this),off:this.off.bind(this),emit:this.emit.bind(this)},hiddenDates:[],util:{snap:null,toScreen:h._toScreen.bind(h),toGlobalScreen:h._toGlobalScreen.bind(h),toTime:h._toTime.bind(h),toGlobalTime:h._toGlobalTime.bind(h)}},this.range=new a(this.body),this.components.push(this.range),this.body.range=this.range,this.timeAxis=new d(this.body),this.components.push(this.timeAxis),this.body.util.snap=this.timeAxis.snap.bind(this.timeAxis),this.currentTime=new l(this.body),this.components.push(this.currentTime),this.customTime=new c(this.body),this.components.push(this.customTime),this.linegraph=new p(this.body),this.components.push(this.linegraph),this.itemsData=null,this.groupsData=null,s&&this.setOptions(s),i&&this.setGroups(i),e?this.setItems(e):this.redraw()}var o=(i(56),i(45),i(1)),n=i(3),r=i(4),a=i(17),h=i(46),d=i(30),l=i(21),c=i(22),p=i(29);s.prototype=new h,s.prototype.setItems=function(t){var e,i=null==this.itemsData;if(e=t?t instanceof n||t instanceof r?t:new n(t,{type:{start:"Date",end:"Date"}}):null,this.itemsData=e,this.linegraph&&this.linegraph.setItems(e),i)if(void 0!=this.options.start||void 0!=this.options.end){var s=void 0!=this.options.start?this.options.start:null,o=void 0!=this.options.end?this.options.end:null;this.setWindow(s,o,{animate:!1})}else this.fit({animate:!1})},s.prototype.setGroups=function(t){var e;e=t?t instanceof n||t instanceof r?t:new n(t):null,this.groupsData=e,this.linegraph.setGroups(e)},s.prototype.getLegend=function(t,e,i){return void 0===e&&(e=15),void 0===i&&(i=15),void 0!==this.linegraph.groups[t]?this.linegraph.groups[t].getLegend(e,i):"cannot find group:"+t},s.prototype.isGroupVisible=function(t){return void 0!==this.linegraph.groups[t]?this.linegraph.groups[t].visible&&(void 0===this.linegraph.options.groups.visibility[t]||1==this.linegraph.options.groups.visibility[t]):!1},s.prototype.getItemRange=function(){var t=null,e=null;for(var i in this.linegraph.groups)if(this.linegraph.groups.hasOwnProperty(i)&&1==this.linegraph.groups[i].visible)for(var s=0;sr?r:t,e=null==e?r:r>e?r:e}return{min:null!=t?new Date(t):null,max:null!=e?new Date(e):null}},t.exports=s},function(t,e,i){var s=i(44);e.convertHiddenOptions=function(t,e){if(t.hiddenDates=[],e&&1==Array.isArray(e)){for(var i=0;i=4*a){var p=0,u=n.clone();switch(i[h].repeat){case"daily":d.day()!=l.day()&&(p=1),d.dayOfYear(o.dayOfYear()),d.year(o.year()),d.subtract(7,"days"),l.dayOfYear(o.dayOfYear()),l.year(o.year()),l.subtract(7-p,"days"),u.add(1,"weeks");break;case"weekly":var m=l.diff(d,"days"),f=d.day();d.date(o.date()),d.month(o.month()),d.year(o.year()),l=d.clone(),d.day(f),l.day(f),l.add(m,"days"),d.subtract(1,"weeks"),l.subtract(1,"weeks"),u.add(1,"weeks");break;case"monthly":d.month()!=l.month()&&(p=1),d.month(o.month()),d.year(o.year()),d.subtract(1,"months"),l.month(o.month()),l.year(o.year()),l.subtract(1,"months"),l.add(p,"months"),u.add(1,"months");break;case"yearly":d.year()!=l.year()&&(p=1),d.year(o.year()),d.subtract(1,"years"),l.year(o.year()),l.subtract(1,"years"),l.add(p,"years"),u.add(1,"years");break;default:return void console.log("Wrong repeat format, allowed are: daily, weekly, monthly, yearly. Given:",i[h].repeat)}for(;u>d;)switch(t.hiddenDates.push({start:d.valueOf(),end:l.valueOf()}),i[h].repeat){case"daily":d.add(1,"days"),l.add(1,"days");break;case"weekly":d.add(1,"weeks"),l.add(1,"weeks");break;case"monthly":d.add(1,"months"),l.add(1,"months");break;case"yearly":d.add(1,"y"),l.add(1,"y");break;default:return void console.log("Wrong repeat format, allowed are: daily, weekly, monthly, yearly. Given:",i[h].repeat)}t.hiddenDates.push({start:d.valueOf(),end:l.valueOf()})}}e.removeDuplicates(t);var g=e.isHidden(t.range.start,t.hiddenDates),v=e.isHidden(t.range.end,t.hiddenDates),y=t.range.start,b=t.range.end;1==g.hidden&&(y=1==t.range.startToFront?g.startDate-1:g.endDate+1),1==v.hidden&&(b=1==t.range.endToFront?v.startDate-1:v.endDate+1),(1==g.hidden||1==v.hidden)&&t.range._applyRange(y,b)}},e.removeDuplicates=function(t){for(var e=t.hiddenDates,i=[],s=0;s=e[s].start&&e[o].end<=e[s].end?e[o].remove=!0:e[o].start>=e[s].start&&e[o].start<=e[s].end?(e[s].end=e[o].end,e[o].remove=!0):e[o].end>=e[s].start&&e[o].end<=e[s].end&&(e[s].start=e[o].start,e[o].remove=!0));for(var s=0;s=r&&a>o){i=!0;break}}if(1==i&&o=e&&i>r&&(s+=r-n)}return s},e.correctTimeForHidden=function(t,i,o){return o=s(o).toDate().valueOf(),o-=e.getHiddenDurationBefore(t,i,o)},e.getHiddenDurationBefore=function(t,e,i){var o=0;i=s(i).toDate().valueOf();for(var n=0;n=e.start&&a=a&&(o+=a-r)}return o},e.getAccumulatedHiddenDuration=function(t,e,i){for(var s=0,o=0,n=e.start,r=0;r=e.start&&h=i)break;s+=h-a}}return s},e.snapAwayFromHidden=function(t,i,s,o){var n=e.isHidden(i,t);return 1==n.hidden?0>s?1==o?n.startDate-(n.endDate-i)-1:n.startDate-1:1==o?n.endDate+(i-n.startDate)+1:n.endDate+1:i},e.isHidden=function(t,e){for(var i=0;i=s&&o>t)return{hidden:!0,startDate:s,endDate:o}}return{hidden:!1,startDate:s,endDate:o}}},function(t){function e(t,e,i,s,o,n){this.current=0,this.autoScale=!0,this.stepIndex=0,this.step=1,this.scale=1,this.marginStart,this.marginEnd,this.deadSpace=0,this.majorSteps=[1,2,5,10],this.minorSteps=[.25,.5,1,2],this.alignZeros=n,this.setRange(t,e,i,s,o)}e.prototype.setRange=function(t,e,i,s,o){this._start=void 0===o.min?t:o.min,this._end=void 0===o.max?e:o.max,this._start==this._end&&(this._start-=.75,this._end+=1),1==this.autoScale&&this.setMinimumStep(i,s),this.setFirst(o)},e.prototype.setMinimumStep=function(t,e){var i=this._end-this._start,s=1.2*i,o=t*(s/e),n=Math.round(Math.log(s)/Math.LN10),r=-1,a=Math.pow(10,n),h=0;0>n&&(h=n);for(var d=!1,l=h;Math.abs(l)<=Math.abs(n);l++){a=Math.pow(10,l);for(var c=0;c=o){d=!0,r=c;break}}if(1==d)break}this.stepIndex=r,this.scale=a,this.step=a*this.minorSteps[r]},e.prototype.setFirst=function(t){void 0===t&&(t={});var e=void 0===t.min?this._start-2*this.scale*this.minorSteps[this.stepIndex]:t.min,i=void 0===t.max?this._end+this.scale*this.minorSteps[this.stepIndex]:t.max;this.marginEnd=void 0===t.max?this.roundToMinor(i):t.max,this.marginStart=void 0===t.min?this.roundToMinor(e):t.min,1==this.alignZeros&&(this.marginEnd-this.marginStart)%this.step!=0&&(this.marginEnd+=this.marginEnd%this.step),this.deadSpace=this.roundToMinor(i)-i+this.roundToMinor(e)-e,this.marginRange=this.marginEnd-this.marginStart,this.current=this.marginEnd},e.prototype.roundToMinor=function(t){var e=t-t%(this.scale*this.minorSteps[this.stepIndex]);return t%(this.scale*this.minorSteps[this.stepIndex])>.5*this.scale*this.minorSteps[this.stepIndex]?e+this.scale*this.minorSteps[this.stepIndex]:e},e.prototype.hasNext=function(){return this.current>=this.marginStart},e.prototype.next=function(){var t=this.current;this.current-=this.step,this.current==t&&(this.current=this._end)},e.prototype.previous=function(){this.current+=this.step,this.marginEnd+=this.step,this.marginRange=this.marginEnd-this.marginStart},e.prototype.getCurrent=function(t){var e=Math.abs(this.current)0;s--){if("0"!=i[s]){if("."==i[s]||","==i[s]){i=i.slice(0,s);break}break}i=i.slice(0,s)}}else{var o="",n=i.indexOf("e");if(-1!=n&&(o=i.slice(n),i=i.slice(0,n)),n=Math.max(i.indexOf(","),i.indexOf(".")),-1===n?(0!==t&&(i+="."),n=i.length+t):0!==t&&(n+=t+1),n>i.length)for(var r=n-i.length;r>0;r--)i+="0";else i=i.slice(0,n);i+=o}return i},e.prototype.snap=function(){},e.prototype.isMajor=function(){return this.current%(this.scale*this.majorSteps[this.stepIndex])==0},t.exports=e},function(t,e,i){function s(t,e){var i=a().hours(0).minutes(0).seconds(0).milliseconds(0);this.start=i.clone().add(-3,"days").valueOf(),this.end=i.clone().add(4,"days").valueOf(),this.body=t,this.deltaDifference=0,this.scaleOffset=0,this.startToFront=!1,this.endToFront=!0,this.defaultOptions={start:null,end:null,direction:"horizontal",moveable:!0,zoomable:!0,min:null,max:null,zoomMin:10,zoomMax:31536e10},this.options=r.extend({},this.defaultOptions),this.props={touch:{}},this.animateTimer=null,this.body.emitter.on("panstart",this._onDragStart.bind(this)),this.body.emitter.on("panmove",this._onDrag.bind(this)),this.body.emitter.on("panend",this._onDragEnd.bind(this)),this.body.emitter.on("press",this._onHold.bind(this)),this.body.emitter.on("mousewheel",this._onMouseWheel.bind(this)),this.body.emitter.on("touch",this._onTouch.bind(this)),this.body.emitter.on("pinch",this._onPinch.bind(this)),this.setOptions(e)}function o(t){if("horizontal"!=t&&"vertical"!=t)throw new TypeError('Unknown direction "'+t+'". Choose "horizontal" or "vertical".')}function n(t,e){return{x:t.x-r.getAbsoluteLeft(e),y:t.y-r.getAbsoluteTop(e)}}var r=i(1),a=(i(47),i(44)),h=i(20),d=i(15);s.prototype=new h,s.prototype.setOptions=function(t){if(t){var e=["direction","min","max","zoomMin","zoomMax","moveable","zoomable","activate","hiddenDates"];r.selectiveExtend(e,this.options,t),("start"in t||"end"in t)&&this.setRange(t.start,t.end)}},s.prototype.setRange=function(t,e,i,s){s!==!0&&(s=!1);var o=void 0!=t?r.convert(t,"Date").valueOf():null,n=void 0!=e?r.convert(e,"Date").valueOf():null;if(this._cancelAnimation(),i){var a=this,h=this.start,l=this.end,c="number"==typeof i?i:500,p=(new Date).valueOf(),u=!1,m=function(){if(!a.props.touch.dragging){var t=(new Date).valueOf(),e=t-p,i=e>c,g=i||null===o?o:r.easeInOutQuad(e,h,o,c),v=i||null===n?n:r.easeInOutQuad(e,l,n,c);f=a._applyRange(g,v),d.updateHiddenDates(a.body,a.options.hiddenDates),u=u||f,f&&a.body.emitter.emit("rangechange",{start:new Date(a.start),end:new Date(a.end),byUser:s}),i?u&&a.body.emitter.emit("rangechanged",{start:new Date(a.start),end:new Date(a.end),byUser:s}):a.animateTimer=setTimeout(m,20)}};return m()}var f=this._applyRange(o,n);if(d.updateHiddenDates(this.body,this.options.hiddenDates),f){var g={start:new Date(this.start),end:new Date(this.end),byUser:s};this.body.emitter.emit("rangechange",g),this.body.emitter.emit("rangechanged",g)}},s.prototype._cancelAnimation=function(){this.animateTimer&&(clearTimeout(this.animateTimer),this.animateTimer=null)},s.prototype._applyRange=function(t,e){var i,s=null!=t?r.convert(t,"Date").valueOf():this.start,o=null!=e?r.convert(e,"Date").valueOf():this.end,n=null!=this.options.max?r.convert(this.options.max,"Date").valueOf():null,a=null!=this.options.min?r.convert(this.options.min,"Date").valueOf():null;if(isNaN(s)||null===s)throw new Error('Invalid start "'+t+'"');if(isNaN(o)||null===o)throw new Error('Invalid end "'+e+'"');if(s>o&&(o=s),null!==a&&a>s&&(i=a-s,s+=i,o+=i,null!=n&&o>n&&(o=n)),null!==n&&o>n&&(i=o-n,s-=i,o-=i,null!=a&&a>s&&(s=a)),null!==this.options.zoomMin){var h=parseFloat(this.options.zoomMin);0>h&&(h=0),h>o-s&&(this.end-this.start===h?(s=this.start,o=this.end):(i=h-(o-s),s-=i/2,o+=i/2))}if(null!==this.options.zoomMax){var d=parseFloat(this.options.zoomMax);0>d&&(d=0),o-s>d&&(this.end-this.start===d?(s=this.start,o=this.end):(i=o-s-d,s+=i/2,o-=i/2))}var l=this.start!=s||this.end!=o;return s>=this.start&&s<=this.end||o>=this.start&&o<=this.end||this.start>=s&&this.start<=o||this.end>=s&&this.end<=o||this.body.emitter.emit("checkRangedItems"),this.start=s,this.end=o,l},s.prototype.getRange=function(){return{start:this.start,end:this.end}},s.prototype.conversion=function(t,e){return s.conversion(this.start,this.end,t,e)},s.conversion=function(t,e,i,s){return void 0===s&&(s=0),0!=i&&e-t!=0?{offset:t,scale:i/(e-t-s)}:{offset:0,scale:1}},s.prototype._onDragStart=function(t){this.deltaDifference=0,this.previousDelta=0,this.options.moveable&&this.props.touch.allowDragging&&(this.props.touch.start=this.start,this.props.touch.end=this.end,this.props.touch.dragging=!0,this.body.dom.root&&(this.body.dom.root.style.cursor="move"),t.preventDefault())},s.prototype._onDrag=function(t){if(this.options.moveable&&this.props.touch.allowDragging){var e=this.options.direction;o(e);var i="horizontal"==e?t.deltaX:t.deltaY;i-=this.deltaDifference;var s=this.props.touch.end-this.props.touch.start,n=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end);s-=n;var r="horizontal"==e?this.body.domProps.center.width:this.body.domProps.center.height,a=-i/r*s,h=this.props.touch.start+a,l=this.props.touch.end+a,c=d.snapAwayFromHidden(this.body.hiddenDates,h,this.previousDelta-i,!0),p=d.snapAwayFromHidden(this.body.hiddenDates,l,this.previousDelta-i,!0);if(c!=h||p!=l)return this.deltaDifference+=i,this.props.touch.start=c,this.props.touch.end=p,void this._onDrag(t);this.previousDelta=i,this._applyRange(h,l),this.body.emitter.emit("rangechange",{start:new Date(this.start),end:new Date(this.end),byUser:!0}),t.preventDefault()}},s.prototype._onDragEnd=function(){this.options.moveable&&this.props.touch.allowDragging&&(this.props.touch.dragging=!1,this.body.dom.root&&(this.body.dom.root.style.cursor="auto"),this.body.emitter.emit("rangechanged",{start:new Date(this.start),end:new Date(this.end),byUser:!0}))},s.prototype._onMouseWheel=function(t){if(this.options.zoomable&&this.options.moveable){var e=0;if(t.wheelDelta?e=t.wheelDelta/120:t.detail&&(e=-t.detail/3),e){var i;i=0>e?1-e/5:1/(1+e/5);var s=n({x:t.pageX,y:t.pageY},this.body.dom.center),o=this._pointerToDate(s);this.zoom(i,o,e)}t.preventDefault()}},s.prototype._onTouch=function(){this.props.touch.start=this.start,this.props.touch.end=this.end,this.props.touch.allowDragging=!0,this.props.touch.center=null,this.scaleOffset=0,this.deltaDifference=0},s.prototype._onHold=function(){this.props.touch.allowDragging=!1},s.prototype._onPinch=function(t){if(this.options.zoomable&&this.options.moveable){this.props.touch.allowDragging=!1,this.props.touch.center||(this.props.touch.center=n(t.center,this.body.dom.center));var e=1/(t.scale+this.scaleOffset),i=this._pointerToDate(this.props.touch.center),s=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end),o=d.getHiddenDurationBefore(this.body.hiddenDates,this,i),r=s-o,a=i-o+(this.props.touch.start-(i-o))*e,h=i+r+(this.props.touch.end-(i+r))*e;this.startToFront=0>=1-e,this.endToFront=0>=e-1;var l=d.snapAwayFromHidden(this.body.hiddenDates,a,1-e,!0),c=d.snapAwayFromHidden(this.body.hiddenDates,h,e-1,!0);(l!=a||c!=h)&&(this.props.touch.start=l,this.props.touch.end=c,this.scaleOffset=1-t.scale,a=l,h=c),this.setRange(a,h,!1,!0),this.startToFront=!1,this.endToFront=!0,t.preventDefault()}},s.prototype._pointerToDate=function(t){var e,i=this.options.direction;if(o(i),"horizontal"==i)return this.body.util.toTime(t.x).valueOf();var s=this.body.domProps.center.height;return e=this.conversion(s),t.y/e.scale+e.offset},s.prototype.zoom=function(t,e,i){null==e&&(e=(this.start+this.end)/2);var s=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end),o=d.getHiddenDurationBefore(this.body.hiddenDates,this,e),n=s-o,r=e-o+(this.start-(e-o))*t,a=e+n+(this.end-(e+n))*t;this.startToFront=i>0?!1:!0,this.endToFront=-i>0?!1:!0;var h=d.snapAwayFromHidden(this.body.hiddenDates,r,i,!0),l=d.snapAwayFromHidden(this.body.hiddenDates,a,-i,!0);(h!=r||l!=a)&&(r=h,a=l),this.setRange(r,a,!1,!0),this.startToFront=!1,this.endToFront=!0},s.prototype.move=function(t){var e=this.end-this.start,i=this.start+e*t,s=this.end+e*t;this.start=i,this.end=s},s.prototype.moveTo=function(t){var e=(this.start+this.end)/2,i=e-t,s=this.start-i,o=this.end-i;this.setRange(s,o)},t.exports=s},function(t,e){var i=.001;e.orderByStart=function(t){t.sort(function(t,e){return t.data.start-e.data.start})},e.orderByEnd=function(t){t.sort(function(t,e){var i="end"in t.data?t.data.end:t.data.start,s="end"in e.data?e.data.end:e.data.start;return i-s})},e.stack=function(t,i,s){var o,n;if(s)for(o=0,n=t.length;n>o;o++)t[o].top=null;for(o=0,n=t.length;n>o;o++){var r=t[o];if(r.stack&&null===r.top){r.top=i.axis;do{for(var a=null,h=0,d=t.length;d>h;h++){var l=t[h];if(null!==l.top&&l!==r&&l.stack&&e.collision(r,l,i.item)){a=l;break}}null!=a&&(r.top=a.top+a.height+i.item.vertical)}while(a)}}},e.nostack=function(t,e,i){var s,o,n;for(s=0,o=t.length;o>s;s++)if(void 0!==t[s].data.subgroup){n=e.axis;for(var r in i)i.hasOwnProperty(r)&&1==i[r].visible&&i[r].indexe.left&&t.top-s.vertical+ie.top}},function(t,e,i){function s(t,e,i,o){this.current=new Date,this._start=new Date,this._end=new Date,this.autoScale=!0,this.scale="day",this.step=1,this.setRange(t,e,i),this.switchedDay=!1,this.switchedMonth=!1,this.switchedYear=!1,this.hiddenDates=o,void 0===o&&(this.hiddenDates=[]),this.format=s.FORMAT}var o=i(44),n=i(15),r=i(1);s.FORMAT={minorLabels:{millisecond:"SSS",second:"s",minute:"HH:mm",hour:"HH:mm",weekday:"ddd D",day:"D",month:"MMM",year:"YYYY"},majorLabels:{millisecond:"HH:mm:ss",second:"D MMMM HH:mm",minute:"ddd D MMMM",hour:"ddd D MMMM",weekday:"MMMM YYYY",day:"MMMM YYYY",month:"YYYY",year:""}},s.prototype.setFormat=function(t){var e=r.deepExtend({},s.FORMAT);this.format=r.deepExtend(e,t)},s.prototype.setRange=function(t,e,i){if(!(t instanceof Date&&e instanceof Date))throw"No legal start or end date in method setRange";this._start=void 0!=t?new Date(t.valueOf()):new Date,this._end=void 0!=e?new Date(e.valueOf()):new Date,this.autoScale&&this.setMinimumStep(i)},s.prototype.first=function(){this.current=new Date(this._start.valueOf()),this.roundToMinor()},s.prototype.roundToMinor=function(){switch(this.scale){case"year":this.current.setFullYear(this.step*Math.floor(this.current.getFullYear()/this.step)),this.current.setMonth(0);case"month":this.current.setDate(1);case"day":case"weekday":this.current.setHours(0);case"hour":this.current.setMinutes(0);case"minute":this.current.setSeconds(0);case"second":this.current.setMilliseconds(0)}if(1!=this.step)switch(this.scale){case"millisecond":this.current.setMilliseconds(this.current.getMilliseconds()-this.current.getMilliseconds()%this.step);break;case"second":this.current.setSeconds(this.current.getSeconds()-this.current.getSeconds()%this.step);break;case"minute":this.current.setMinutes(this.current.getMinutes()-this.current.getMinutes()%this.step);break;case"hour":this.current.setHours(this.current.getHours()-this.current.getHours()%this.step);break;case"weekday":case"day":this.current.setDate(this.current.getDate()-1-(this.current.getDate()-1)%this.step+1);break;case"month":this.current.setMonth(this.current.getMonth()-this.current.getMonth()%this.step);break;case"year":this.current.setFullYear(this.current.getFullYear()-this.current.getFullYear()%this.step)}},s.prototype.hasNext=function(){return this.current.valueOf()<=this._end.valueOf()},s.prototype.next=function(){var t=this.current.valueOf();if(this.current.getMonth()<6)switch(this.scale){case"millisecond":this.current=new Date(this.current.valueOf()+this.step);break;case"second":this.current=new Date(this.current.valueOf()+1e3*this.step);break;case"minute":this.current=new Date(this.current.valueOf()+1e3*this.step*60);break;case"hour":this.current=new Date(this.current.valueOf()+1e3*this.step*60*60);var e=this.current.getHours();this.current.setHours(e-e%this.step);break;case"weekday":case"day":this.current.setDate(this.current.getDate()+this.step);break;case"month":this.current.setMonth(this.current.getMonth()+this.step);break;case"year":this.current.setFullYear(this.current.getFullYear()+this.step)}else switch(this.scale){case"millisecond":this.current=new Date(this.current.valueOf()+this.step);break;case"second":this.current.setSeconds(this.current.getSeconds()+this.step);break;case"minute":this.current.setMinutes(this.current.getMinutes()+this.step); -break;case"hour":this.current.setHours(this.current.getHours()+this.step);break;case"weekday":case"day":this.current.setDate(this.current.getDate()+this.step);break;case"month":this.current.setMonth(this.current.getMonth()+this.step);break;case"year":this.current.setFullYear(this.current.getFullYear()+this.step)}if(1!=this.step)switch(this.scale){case"millisecond":this.current.getMilliseconds()0&&(this.step=e),this.autoScale=!1},s.prototype.setAutoScale=function(t){this.autoScale=t},s.prototype.setMinimumStep=function(t){if(void 0!=t){var e=31104e6,i=2592e6,s=864e5,o=36e5,n=6e4,r=1e3,a=1;1e3*e>t&&(this.scale="year",this.step=1e3),500*e>t&&(this.scale="year",this.step=500),100*e>t&&(this.scale="year",this.step=100),50*e>t&&(this.scale="year",this.step=50),10*e>t&&(this.scale="year",this.step=10),5*e>t&&(this.scale="year",this.step=5),e>t&&(this.scale="year",this.step=1),3*i>t&&(this.scale="month",this.step=3),i>t&&(this.scale="month",this.step=1),5*s>t&&(this.scale="day",this.step=5),2*s>t&&(this.scale="day",this.step=2),s>t&&(this.scale="day",this.step=1),s/2>t&&(this.scale="weekday",this.step=1),4*o>t&&(this.scale="hour",this.step=4),o>t&&(this.scale="hour",this.step=1),15*n>t&&(this.scale="minute",this.step=15),10*n>t&&(this.scale="minute",this.step=10),5*n>t&&(this.scale="minute",this.step=5),n>t&&(this.scale="minute",this.step=1),15*r>t&&(this.scale="second",this.step=15),10*r>t&&(this.scale="second",this.step=10),5*r>t&&(this.scale="second",this.step=5),r>t&&(this.scale="second",this.step=1),200*a>t&&(this.scale="millisecond",this.step=200),100*a>t&&(this.scale="millisecond",this.step=100),50*a>t&&(this.scale="millisecond",this.step=50),10*a>t&&(this.scale="millisecond",this.step=10),5*a>t&&(this.scale="millisecond",this.step=5),a>t&&(this.scale="millisecond",this.step=1)}},s.prototype.snap=function(t){var e=new Date(t.valueOf());if("year"==this.scale){var i=e.getFullYear()+Math.round(e.getMonth()/12);e.setFullYear(Math.round(i/this.step)*this.step),e.setMonth(0),e.setDate(0),e.setHours(0),e.setMinutes(0),e.setSeconds(0),e.setMilliseconds(0)}else if("month"==this.scale)e.getDate()>15?(e.setDate(1),e.setMonth(e.getMonth()+1)):e.setDate(1),e.setHours(0),e.setMinutes(0),e.setSeconds(0),e.setMilliseconds(0);else if("day"==this.scale){switch(this.step){case 5:case 2:e.setHours(24*Math.round(e.getHours()/24));break;default:e.setHours(12*Math.round(e.getHours()/12))}e.setMinutes(0),e.setSeconds(0),e.setMilliseconds(0)}else if("weekday"==this.scale){switch(this.step){case 5:case 2:e.setHours(12*Math.round(e.getHours()/12));break;default:e.setHours(6*Math.round(e.getHours()/6))}e.setMinutes(0),e.setSeconds(0),e.setMilliseconds(0)}else if("hour"==this.scale){switch(this.step){case 4:e.setMinutes(60*Math.round(e.getMinutes()/60));break;default:e.setMinutes(30*Math.round(e.getMinutes()/30))}e.setSeconds(0),e.setMilliseconds(0)}else if("minute"==this.scale){switch(this.step){case 15:case 10:e.setMinutes(5*Math.round(e.getMinutes()/5)),e.setSeconds(0);break;case 5:e.setSeconds(60*Math.round(e.getSeconds()/60));break;default:e.setSeconds(30*Math.round(e.getSeconds()/30))}e.setMilliseconds(0)}else if("second"==this.scale)switch(this.step){case 15:case 10:e.setSeconds(5*Math.round(e.getSeconds()/5)),e.setMilliseconds(0);break;case 5:e.setMilliseconds(1e3*Math.round(e.getMilliseconds()/1e3));break;default:e.setMilliseconds(500*Math.round(e.getMilliseconds()/500))}else if("millisecond"==this.scale){var s=this.step>5?this.step/2:1;e.setMilliseconds(Math.round(e.getMilliseconds()/s)*s)}return e},s.prototype.isMajor=function(){if(1==this.switchedYear)switch(this.switchedYear=!1,this.scale){case"year":case"month":case"weekday":case"day":case"hour":case"minute":case"second":case"millisecond":return!0;default:return!1}else if(1==this.switchedMonth)switch(this.switchedMonth=!1,this.scale){case"weekday":case"day":case"hour":case"minute":case"second":case"millisecond":return!0;default:return!1}else if(1==this.switchedDay)switch(this.switchedDay=!1,this.scale){case"millisecond":case"second":case"minute":case"hour":return!0;default:return!1}switch(this.scale){case"millisecond":return 0==this.current.getMilliseconds();case"second":return 0==this.current.getSeconds();case"minute":return 0==this.current.getHours()&&0==this.current.getMinutes();case"hour":return 0==this.current.getHours();case"weekday":case"day":return 1==this.current.getDate();case"month":return 0==this.current.getMonth();case"year":return!1;default:return!1}},s.prototype.getLabelMinor=function(t){void 0==t&&(t=this.current);var e=this.format.minorLabels[this.scale];return e&&e.length>0?o(t).format(e):""},s.prototype.getLabelMajor=function(t){void 0==t&&(t=this.current);var e=this.format.majorLabels[this.scale];return e&&e.length>0?o(t).format(e):""},s.prototype.getClassName=function(){function t(t){return t/h%2==0?" even":" odd"}function e(t){return t.isSame(new Date,"day")?" today":t.isSame(o().add(1,"day"),"day")?" tomorrow":t.isSame(o().add(-1,"day"),"day")?" yesterday":""}function i(t){return t.isSame(new Date,"week")?" current-week":""}function s(t){return t.isSame(new Date,"month")?" current-month":""}function n(t){return t.isSame(new Date,"year")?" current-year":""}var r=o(this.current),a=r.locale?r.locale("en"):r.lang("en"),h=this.step;switch(this.scale){case"millisecond":return t(a.milliseconds()).trim();case"second":return t(a.seconds()).trim();case"minute":return t(a.minutes()).trim();case"hour":var d=a.hours();return 4==this.step&&(d=d+"-"+(d+4)),d+"h"+e(a)+t(a.hours());case"weekday":return a.format("dddd").toLowerCase()+e(a)+i(a)+t(a.date());case"day":var l=a.date(),c=a.format("MMMM").toLowerCase();return"day"+l+" "+c+s(a)+t(l-1);case"month":return a.format("MMMM").toLowerCase()+s(a)+t(a.month());case"year":var p=a.year();return"year"+p+n(a)+t(p);default:return""}},t.exports=s},function(t){function e(){this.options=null,this.props=null}e.prototype.setOptions=function(t){t&&util.extend(this.options,t)},e.prototype.redraw=function(){return!1},e.prototype.destroy=function(){},e.prototype._isResized=function(){var t=this.props._previousWidth!==this.props.width||this.props._previousHeight!==this.props.height;return this.props._previousWidth=this.props.width,this.props._previousHeight=this.props.height,t},t.exports=e},function(t,e,i){function s(t,e){this.body=t,this.defaultOptions={showCurrentTime:!0,locales:a,locale:"en"},this.options=o.extend({},this.defaultOptions),this.offset=0,this._create(),this.setOptions(e)}var o=i(1),n=i(20),r=i(44),a=i(48);s.prototype=new n,s.prototype._create=function(){var t=document.createElement("div");t.className="currenttime",t.style.position="absolute",t.style.top="0px",t.style.height="100%",this.bar=t},s.prototype.destroy=function(){this.options.showCurrentTime=!1,this.redraw(),this.body=null},s.prototype.setOptions=function(t){t&&o.selectiveExtend(["showCurrentTime","locale","locales"],this.options,t)},s.prototype.redraw=function(){if(this.options.showCurrentTime){var t=this.body.dom.backgroundVertical;this.bar.parentNode!=t&&(this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),t.appendChild(this.bar),this.start());var e=new Date((new Date).valueOf()+this.offset),i=this.body.util.toScreen(e),s=this.options.locales[this.options.locale],o=s.current+" "+s.time+": "+r(e).format("dddd, MMMM Do YYYY, H:mm:ss");o=o.charAt(0).toUpperCase()+o.substring(1),this.bar.style.left=i+"px",this.bar.title=o}else this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),this.stop();return!1},s.prototype.start=function(){function t(){e.stop();var i=e.body.range.conversion(e.body.domProps.center.width).scale,s=1/i/10;30>s&&(s=30),s>1e3&&(s=1e3),e.redraw(),e.currentTimeTimer=setTimeout(t,s)}var e=this;t()},s.prototype.stop=function(){void 0!==this.currentTimeTimer&&(clearTimeout(this.currentTimeTimer),delete this.currentTimeTimer)},s.prototype.setCurrentTime=function(t){var e=o.convert(t,"Date").valueOf(),i=(new Date).valueOf();this.offset=e-i,this.redraw()},s.prototype.getCurrentTime=function(){return new Date((new Date).valueOf()+this.offset)},t.exports=s},function(t,e,i){function s(t,e){this.body=t,this.defaultOptions={showCustomTime:!1,locales:h,locale:"en"},this.options=n.extend({},this.defaultOptions),this.customTime=new Date,this.eventParams={},this._create(),this.setOptions(e)}var o=i(45),n=i(1),r=i(20),a=i(44),h=i(48);s.prototype=new r,s.prototype.setOptions=function(t){t&&n.selectiveExtend(["showCustomTime","locale","locales"],this.options,t)},s.prototype._create=function(){var t=document.createElement("div");t.className="customtime",t.style.position="absolute",t.style.top="0px",t.style.height="100%",this.bar=t;var e=document.createElement("div");e.style.position="relative",e.style.top="0px",e.style.left="-10px",e.style.height="100%",e.style.width="20px",t.appendChild(e),this.hammer=new o(e),this.hammer.on("panstart",this._onDragStart.bind(this)),this.hammer.on("panmove",this._onDrag.bind(this)),this.hammer.on("panend",this._onDragEnd.bind(this)),this.hammer.on("pan",function(t){t.preventDefault()})},s.prototype.destroy=function(){this.options.showCustomTime=!1,this.redraw(),this.hammer.enable(!1),this.hammer=null,this.body=null},s.prototype.redraw=function(){if(this.options.showCustomTime){var t=this.body.dom.backgroundVertical;this.bar.parentNode!=t&&(this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),t.appendChild(this.bar));var e=this.body.util.toScreen(this.customTime),i=this.options.locales[this.options.locale],s=i.time+": "+a(this.customTime).format("dddd, MMMM Do YYYY, H:mm:ss");s=s.charAt(0).toUpperCase()+s.substring(1),this.bar.style.left=e+"px",this.bar.title=s}else this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar);return!1},s.prototype.setCustomTime=function(t){this.customTime=n.convert(t,"Date"),this.redraw()},s.prototype.getCustomTime=function(){return new Date(this.customTime.valueOf())},s.prototype._onDragStart=function(t){this.eventParams.dragging=!0,this.eventParams.customTime=this.customTime,t.stopPropagation(),t.preventDefault()},s.prototype._onDrag=function(t){if(this.eventParams.dragging){var e=this.body.util.toScreen(this.eventParams.customTime)+t.deltaX,i=this.body.util.toTime(e);this.setCustomTime(i),this.body.emitter.emit("timechange",{time:new Date(this.customTime.valueOf())}),t.stopPropagation(),t.preventDefault()}},s.prototype._onDragEnd=function(t){this.eventParams.dragging&&(this.body.emitter.emit("timechanged",{time:new Date(this.customTime.valueOf())}),t.stopPropagation(),t.preventDefault())},t.exports=s},function(t,e,i){function s(t,e,i,s){this.id=o.randomUUID(),this.body=t,this.defaultOptions={orientation:"left",showMinorLabels:!0,showMajorLabels:!0,icons:!0,majorLinesOffset:7,minorLinesOffset:4,labelOffsetX:10,labelOffsetY:2,iconWidth:20,width:"40px",visible:!0,alignZeros:!0,customRange:{left:{min:void 0,max:void 0},right:{min:void 0,max:void 0}},title:{left:{text:void 0},right:{text:void 0}},format:{left:{decimals:void 0},right:{decimals:void 0}}},this.linegraphOptions=s,this.linegraphSVG=i,this.props={},this.DOMelements={lines:{},labels:{},title:{}},this.dom={},this.range={start:0,end:0},this.options=o.extend({},this.defaultOptions),this.conversionFactor=1,this.setOptions(e),this.width=Number((""+this.options.width).replace("px","")),this.minWidth=this.width,this.height=this.linegraphSVG.offsetHeight,this.hidden=!1,this.stepPixels=25,this.stepPixelsForced=25,this.zeroCrossing=-1,this.lineOffset=0,this.master=!0,this.svgElements={},this.iconsRemoved=!1,this.groups={},this.amountOfGroups=0,this._create();var n=this;this.body.emitter.on("verticalDrag",function(){n.dom.lineContainer.style.top=n.body.domProps.scrollTop+"px"})}var o=i(1),n=i(2),r=i(20),a=i(16);s.prototype=new r,s.prototype.addGroup=function(t,e){this.groups.hasOwnProperty(t)||(this.groups[t]=e),this.amountOfGroups+=1},s.prototype.updateGroup=function(t,e){this.groups[t]=e},s.prototype.removeGroup=function(t){this.groups.hasOwnProperty(t)&&(delete this.groups[t],this.amountOfGroups-=1)},s.prototype.setOptions=function(t){if(t){var e=!1;this.options.orientation!=t.orientation&&void 0!==t.orientation&&(e=!0);var i=["orientation","showMinorLabels","showMajorLabels","icons","majorLinesOffset","minorLinesOffset","labelOffsetX","labelOffsetY","iconWidth","width","visible","customRange","title","format","alignZeros"];o.selectiveExtend(i,this.options,t),this.minWidth=Number((""+this.options.width).replace("px","")),1==e&&this.dom.frame&&(this.hide(),this.show())}},s.prototype._create=function(){this.dom.frame=document.createElement("div"),this.dom.frame.style.width=this.options.width,this.dom.frame.style.height=this.height,this.dom.lineContainer=document.createElement("div"),this.dom.lineContainer.style.width="100%",this.dom.lineContainer.style.height=this.height,this.dom.lineContainer.style.position="relative",this.svg=document.createElementNS("http://www.w3.org/2000/svg","svg"),this.svg.style.position="absolute",this.svg.style.top="0px",this.svg.style.height="100%",this.svg.style.width="100%",this.svg.style.display="block",this.dom.frame.appendChild(this.svg)},s.prototype._redrawGroupIcons=function(){n.prepareElements(this.svgElements);var t,e=this.options.iconWidth,i=15,s=4,o=s+.5*i;t="left"==this.options.orientation?s:this.width-e-s;for(var r in this.groups)this.groups.hasOwnProperty(r)&&(1!=this.groups[r].visible||void 0!==this.linegraphOptions.visibility[r]&&1!=this.linegraphOptions.visibility[r]||(this.groups[r].drawIcon(t,o,this.svgElements,this.svg,e,i),o+=i+s));n.cleanupElements(this.svgElements),this.iconsRemoved=!1},s.prototype._cleanupIcons=function(){0==this.iconsRemoved&&(n.prepareElements(this.svgElements),n.cleanupElements(this.svgElements),this.iconsRemoved=!0)},s.prototype.show=function(){this.hidden=!1,this.dom.frame.parentNode||("left"==this.options.orientation?this.body.dom.left.appendChild(this.dom.frame):this.body.dom.right.appendChild(this.dom.frame)),this.dom.lineContainer.parentNode||this.body.dom.backgroundHorizontal.appendChild(this.dom.lineContainer)},s.prototype.hide=function(){this.hidden=!0,this.dom.frame.parentNode&&this.dom.frame.parentNode.removeChild(this.dom.frame),this.dom.lineContainer.parentNode&&this.dom.lineContainer.parentNode.removeChild(this.dom.lineContainer)},s.prototype.setRange=function(t,e){0==this.master&&1==this.options.alignZeros&&-1!=this.zeroCrossing&&t>0&&(t=0),this.range.start=t,this.range.end=e},s.prototype.redraw=function(){var t=!1,e=0;this.dom.lineContainer.style.top=this.body.domProps.scrollTop+"px";for(var i in this.groups)this.groups.hasOwnProperty(i)&&(1!=this.groups[i].visible||void 0!==this.linegraphOptions.visibility[i]&&1!=this.linegraphOptions.visibility[i]||e++);if(0==this.amountOfGroups||0==e)this.hide();else{this.show(),this.height=Number(this.linegraphSVG.style.height.replace("px","")),this.dom.lineContainer.style.height=this.height+"px",this.width=1==this.options.visible?Number((""+this.options.width).replace("px","")):0;var s=this.props,o=this.dom.frame;o.className="dataaxis",this._calculateCharSize();var n=this.options.orientation,r=this.options.showMinorLabels,a=this.options.showMajorLabels;s.minorLabelHeight=r?s.minorCharHeight:0,s.majorLabelHeight=a?s.majorCharHeight:0,s.minorLineWidth=this.body.dom.backgroundHorizontal.offsetWidth-this.lineOffset-this.width+2*this.options.minorLinesOffset,s.minorLineHeight=1,s.majorLineWidth=this.body.dom.backgroundHorizontal.offsetWidth-this.lineOffset-this.width+2*this.options.majorLinesOffset,s.majorLineHeight=1,"left"==n?(o.style.top="0",o.style.left="0",o.style.bottom="",o.style.width=this.width+"px",o.style.height=this.height+"px",this.props.width=this.body.domProps.left.width,this.props.height=this.body.domProps.left.height):(o.style.top="",o.style.bottom="0",o.style.left="0",o.style.width=this.width+"px",o.style.height=this.height+"px",this.props.width=this.body.domProps.right.width,this.props.height=this.body.domProps.right.height),t=this._redrawLabels(),t=this._isResized()||t,1==this.options.icons?this._redrawGroupIcons():this._cleanupIcons(),this._redrawTitle(n)}return t},s.prototype._redrawLabels=function(){var t=!1;n.prepareElements(this.DOMelements.lines),n.prepareElements(this.DOMelements.labels);var e=this.options.orientation,i=this.master?this.props.majorCharHeight||10:this.stepPixelsForced,s=new a(this.range.start,this.range.end,i,this.dom.frame.offsetHeight,this.options.customRange[this.options.orientation],0==this.master&&this.options.alignZeros);this.step=s;var o=(this.dom.frame.offsetHeight-s.deadSpace*(this.dom.frame.offsetHeight/s.marginRange))/((s.marginRange-s.deadSpace)/s.step);this.stepPixels=o;var r=this.height/o,h=0;if(0==this.master){o=this.stepPixelsForced,h=Math.round(this.dom.frame.offsetHeight/o-r);for(var d=0;.5*h>d;d++)s.previous();if(r=this.height/o,-1!=this.zeroCrossing&&1==this.options.alignZeros){var l=s.marginEnd/s.step-this.zeroCrossing;if(l>0)for(var d=0;l>d;d++)s.next();else if(0>l)for(var d=0;-l>d;d++)s.previous()}}else r+=.25;this.valueAtZero=s.marginEnd;var c,p=0,u=1;void 0!==this.options.format[e]&&(c=this.options.format[e].decimals),this.maxLabelSize=0;for(var m=0;u=0&&this._redrawLabel(m-2,s.getCurrent(c),e,"yAxis major",this.props.majorCharHeight),this._redrawLine(m,e,"grid horizontal major",this.options.majorLinesOffset,this.props.majorLineWidth)):this._redrawLine(m,e,"grid horizontal minor",this.options.minorLinesOffset,this.props.minorLineWidth),1==this.master&&0==s.current&&(this.zeroCrossing=u),u++}this.conversionFactor=0==this.master?m/(this.valueAtZero-s.current):this.dom.frame.offsetHeight/s.marginRange;var g=0;void 0!==this.options.title[e]&&void 0!==this.options.title[e].text&&(g=this.props.titleCharHeight);var v=1==this.options.icons?Math.max(this.options.iconWidth,g)+this.options.labelOffsetX+15:g+this.options.labelOffsetX+15;return this.maxLabelSize>this.width-v&&1==this.options.visible?(this.width=this.maxLabelSize+v,this.options.width=this.width+"px",n.cleanupElements(this.DOMelements.lines),n.cleanupElements(this.DOMelements.labels),this.redraw(),t=!0):this.maxLabelSizethis.minWidth?(this.width=Math.max(this.minWidth,this.maxLabelSize+v),this.options.width=this.width+"px",n.cleanupElements(this.DOMelements.lines),n.cleanupElements(this.DOMelements.labels),this.redraw(),t=!0):(n.cleanupElements(this.DOMelements.lines),n.cleanupElements(this.DOMelements.labels),t=!1),t},s.prototype.convertValue=function(t){var e=this.valueAtZero-t,i=e*this.conversionFactor;return i},s.prototype._redrawLabel=function(t,e,i,s,o){var r=n.getDOMElement("div",this.DOMelements.labels,this.dom.frame);r.className=s,r.innerHTML=e,"left"==i?(r.style.left="-"+this.options.labelOffsetX+"px",r.style.textAlign="right"):(r.style.right="-"+this.options.labelOffsetX+"px",r.style.textAlign="left"),r.style.top=t-.5*o+this.options.labelOffsetY+"px",e+="";var a=Math.max(this.props.majorCharWidth,this.props.minorCharWidth);this.maxLabelSized;d++){var c=this.visibleItems[d];c.repositionY(e)}return s},s.prototype._calculateHeight=function(t){var e,i=this.visibleItems;this.resetSubgroups();var s=this;if(i.length){var n=i[0].top,r=i[0].top+i[0].height;if(o.forEach(i,function(t){n=Math.min(n,t.top),r=Math.max(r,t.top+t.height),void 0!==t.data.subgroup&&(s.subgroups[t.data.subgroup].height=Math.max(s.subgroups[t.data.subgroup].height,t.height),s.subgroups[t.data.subgroup].visible=!0)}),n>t.axis){var a=n-t.axis;r-=a,o.forEach(i,function(t){t.top-=a})}e=r+t.item.vertical/2}else e=t.axis+t.item.vertical;return e=Math.max(e,this.props.label.height)},s.prototype.show=function(){this.dom.label.parentNode||this.itemSet.dom.labelSet.appendChild(this.dom.label),this.dom.foreground.parentNode||this.itemSet.dom.foreground.appendChild(this.dom.foreground),this.dom.background.parentNode||this.itemSet.dom.background.appendChild(this.dom.background),this.dom.axis.parentNode||this.itemSet.dom.axis.appendChild(this.dom.axis)},s.prototype.hide=function(){var t=this.dom.label;t.parentNode&&t.parentNode.removeChild(t);var e=this.dom.foreground;e.parentNode&&e.parentNode.removeChild(e);var i=this.dom.background;i.parentNode&&i.parentNode.removeChild(i);var s=this.dom.axis;s.parentNode&&s.parentNode.removeChild(s)},s.prototype.add=function(t){if(this.items[t.id]=t,t.setParent(this),void 0!==t.data.subgroup&&(void 0===this.subgroups[t.data.subgroup]&&(this.subgroups[t.data.subgroup]={height:0,visible:!1,index:this.subgroupIndex,items:[]},this.subgroupIndex++),this.subgroups[t.data.subgroup].items.push(t)),this.orderSubgroups(),-1==this.visibleItems.indexOf(t)){var e=this.itemSet.body.range;this._checkIfVisible(t,this.visibleItems,e)}},s.prototype.orderSubgroups=function(){if(void 0!==this.subgroupOrderer){var t=[];if("string"==typeof this.subgroupOrderer){for(var e in this.subgroups)t.push({subgroup:e,sortField:this.subgroups[e].items[0].data[this.subgroupOrderer]});t.sort(function(t,e){return t.sortField-e.sortField})}else if("function"==typeof this.subgroupOrderer){for(var e in this.subgroups)t.push(this.subgroups[e].items[0].data);t.sort(this.subgroupOrderer)}if(t.length>0)for(var i=0;it?-1:l>=t?0:1};if(e.length>0)for(n=0;nl}),1==this.checkRangedItems)for(this.checkRangedItems=!1,n=0;nl})}for(n=0;n=0&&(n=e[r],!o(n));r--)void 0===s[n.id]&&(s[n.id]=!0,i.push(n));for(r=t+1;rs;s++){var n=this.visibleItems[s];n.repositionY(e)}return i},s.prototype.show=function(){this.dom.background.parentNode||this.itemSet.dom.background.appendChild(this.dom.background)},t.exports=s},function(t,e,i){function s(t,e){this.body=t,this.defaultOptions={type:null,orientation:"bottom",align:"auto",stack:!0,groupOrder:null,selectable:!0,editable:{updateTime:!1,updateGroup:!1,add:!1,remove:!1},onAdd:function(t,e){e(t)},onUpdate:function(t,e){e(t)},onMove:function(t,e){e(t)},onRemove:function(t,e){e(t)},onMoving:function(t,e){e(t)},margin:{item:{horizontal:10,vertical:10},axis:20},padding:5},this.options=n.extend({},this.defaultOptions),this.itemOptions={type:{start:"Date",end:"Date"}},this.conversion={toScreen:t.util.toScreen,toTime:t.util.toTime},this.dom={},this.props={},this.hammer=null;var i=this;this.itemsData=null,this.groupsData=null,this.itemListeners={add:function(t,e){i._onAdd(e.items)},update:function(t,e){i._onUpdate(e.items)},remove:function(t,e){i._onRemove(e.items)}},this.groupListeners={add:function(t,e){i._onAddGroups(e.items)},update:function(t,e){i._onUpdateGroups(e.items)},remove:function(t,e){i._onRemoveGroups(e.items)}},this.items={},this.groups={},this.groupIds=[],this.selection=[],this.stackDirty=!0,this.touchParams={},this._create(),this.setOptions(e)}var o=i(45),n=i(1),r=i(3),a=i(4),h=i(20),d=i(25),l=i(26),c=i(33),p=i(34),u=i(35),m=i(32),f="__ungrouped__",g="__background__";s.prototype=new h,s.types={background:m,box:c,range:u,point:p},s.prototype._create=function(){var t=document.createElement("div");t.className="itemset",t["timeline-itemset"]=this,this.dom.frame=t;var e=document.createElement("div");e.className="background",t.appendChild(e),this.dom.background=e;var i=document.createElement("div");i.className="foreground",t.appendChild(i),this.dom.foreground=i;var s=document.createElement("div");s.className="axis",this.dom.axis=s;var n=document.createElement("div");n.className="labelset",this.dom.labelSet=n,this._updateUngrouped();var r=new l(g,null,this);r.show(),this.groups[g]=r,this.hammer=new o(this.body.dom.centerContainer),this.hammer.on("hammer.input",function(t){t.isFirst&&this._onTouch(t)}.bind(this)),this.hammer.on("panstart",this._onDragStart.bind(this)),this.hammer.on("panmove",this._onDrag.bind(this)),this.hammer.on("panend",this._onDragEnd.bind(this)),this.hammer.on("tap",this._onSelectItem.bind(this)),this.hammer.on("press",this._onMultiSelectItem.bind(this)),this.hammer.on("doubletap",this._onAddItem.bind(this)),this.show()},s.prototype.setOptions=function(t){if(t){var e=["type","align","orientation","padding","stack","selectable","groupOrder","dataAttributes","template","hide"];n.selectiveExtend(e,this.options,t),"margin"in t&&("number"==typeof t.margin?(this.options.margin.axis=t.margin,this.options.margin.item.horizontal=t.margin,this.options.margin.item.vertical=t.margin):"object"==typeof t.margin&&(n.selectiveExtend(["axis"],this.options.margin,t.margin),"item"in t.margin&&("number"==typeof t.margin.item?(this.options.margin.item.horizontal=t.margin.item,this.options.margin.item.vertical=t.margin.item):"object"==typeof t.margin.item&&n.selectiveExtend(["horizontal","vertical"],this.options.margin.item,t.margin.item)))),"editable"in t&&("boolean"==typeof t.editable?(this.options.editable.updateTime=t.editable,this.options.editable.updateGroup=t.editable,this.options.editable.add=t.editable,this.options.editable.remove=t.editable):"object"==typeof t.editable&&n.selectiveExtend(["updateTime","updateGroup","add","remove"],this.options.editable,t.editable));var i=function(e){var i=t[e];if(i){if(!(i instanceof Function))throw new Error("option "+e+" must be a function "+e+"(item, callback)");this.options[e]=i}}.bind(this);["onAdd","onUpdate","onRemove","onMove","onMoving"].forEach(i),this.markDirty()}},s.prototype.markDirty=function(){this.groupIds=[],this.stackDirty=!0},s.prototype.destroy=function(){this.hide(),this.setItems(null),this.setGroups(null),this.hammer=null,this.body=null,this.conversion=null},s.prototype.hide=function(){this.dom.frame.parentNode&&this.dom.frame.parentNode.removeChild(this.dom.frame),this.dom.axis.parentNode&&this.dom.axis.parentNode.removeChild(this.dom.axis),this.dom.labelSet.parentNode&&this.dom.labelSet.parentNode.removeChild(this.dom.labelSet)},s.prototype.show=function(){this.dom.frame.parentNode||this.body.dom.center.appendChild(this.dom.frame),this.dom.axis.parentNode||this.body.dom.backgroundVertical.appendChild(this.dom.axis),this.dom.labelSet.parentNode||this.body.dom.left.appendChild(this.dom.labelSet)},s.prototype.setSelection=function(t){var e,i,s,o;for(void 0==t&&(t=[]),Array.isArray(t)||(t=[t]),e=0,i=this.selection.length;i>e;e++)s=this.selection[e],o=this.items[s],o&&o.unselect();for(this.selection=[],e=0,i=t.length;i>e;e++)s=t[e],o=this.items[s],o&&(this.selection.push(s),o.select())},s.prototype.getSelection=function(){return this.selection.concat([])},s.prototype.getVisibleItems=function(){var t=this.body.range.getRange(),e=this.body.util.toScreen(t.start),i=this.body.util.toScreen(t.end),s=[];for(var o in this.groups)if(this.groups.hasOwnProperty(o))for(var n=this.groups[o],r=n.visibleItems,a=0;ae&&s.push(h.id)}return s},s.prototype._deselect=function(t){for(var e=this.selection,i=0,s=e.length;s>i;i++)if(e[i]==t){e.splice(i,1);break}},s.prototype.redraw=function(){var t=this.options.margin,e=this.body.range,i=n.option.asSize,s=this.options,o=s.orientation,r=!1,a=this.dom.frame,h=s.editable.updateTime||s.editable.updateGroup;this.props.top=this.body.domProps.top.height+this.body.domProps.border.top,this.props.left=this.body.domProps.left.width+this.body.domProps.border.left,a.className="itemset"+(h?" editable":""),r=this._orderGroups()||r;var d=e.end-e.start,l=d!=this.lastVisibleInterval||this.props.width!=this.props.lastWidth;l&&(this.stackDirty=!0),this.lastVisibleInterval=d,this.props.lastWidth=this.props.width;var c=this.stackDirty,p=this._firstGroup(),u={item:t.item,axis:t.axis},m={item:t.item,axis:t.item.vertical/2},f=0,v=t.axis+t.item.vertical;return this.groups[g].redraw(e,m,c),n.forEach(this.groups,function(t){var i=t==p?u:m,s=t.redraw(e,i,c);r=s||r,f+=t.height}),f=Math.max(f,v),this.stackDirty=!1,a.style.height=i(f),this.props.width=a.offsetWidth,this.props.height=f,this.dom.axis.style.top=i("top"==o?this.body.domProps.top.height+this.body.domProps.border.top:this.body.domProps.top.height+this.body.domProps.centerContainer.height),this.dom.axis.style.left="0",r=this._isResized()||r},s.prototype._firstGroup=function(){var t="top"==this.options.orientation?0:this.groupIds.length-1,e=this.groupIds[t],i=this.groups[e]||this.groups[f];return i||null},s.prototype._updateUngrouped=function(){{var t,e,i=this.groups[f];this.groups[g]}if(this.groupsData){if(i){i.hide(),delete this.groups[f];for(e in this.items)if(this.items.hasOwnProperty(e)){t=this.items[e],t.parent&&t.parent.remove(t);var s=this._getGroupId(t.data),o=this.groups[s];o&&o.add(t)||t.hide()}}}else if(!i){var n=null,r=null;i=new d(n,r,this),this.groups[f]=i;for(e in this.items)this.items.hasOwnProperty(e)&&(t=this.items[e],i.add(t));i.show()}},s.prototype.getLabelSet=function(){return this.dom.labelSet},s.prototype.setItems=function(t){var e,i=this,s=this.itemsData;if(t){if(!(t instanceof r||t instanceof a))throw new TypeError("Data must be an instance of DataSet or DataView");this.itemsData=t}else this.itemsData=null;if(s&&(n.forEach(this.itemListeners,function(t,e){s.off(e,t)}),e=s.getIds(),this._onRemove(e)),this.itemsData){var o=this.id;n.forEach(this.itemListeners,function(t,e){i.itemsData.on(e,t,o)}),e=this.itemsData.getIds(),this._onAdd(e),this._updateUngrouped()}},s.prototype.getItems=function(){return this.itemsData},s.prototype.setGroups=function(t){var e,i=this;if(this.groupsData&&(n.forEach(this.groupListeners,function(t,e){i.groupsData.unsubscribe(e,t)}),e=this.groupsData.getIds(),this.groupsData=null,this._onRemoveGroups(e)),t){if(!(t instanceof r||t instanceof a))throw new TypeError("Data must be an instance of DataSet or DataView");this.groupsData=t}else this.groupsData=null;if(this.groupsData){var s=this.id;n.forEach(this.groupListeners,function(t,e){i.groupsData.on(e,t,s)}),e=this.groupsData.getIds(),this._onAddGroups(e)}this._updateUngrouped(),this._order(),this.body.emitter.emit("change",{queue:!0})},s.prototype.getGroups=function(){return this.groupsData},s.prototype.removeItem=function(t){var e=this.itemsData.get(t),i=this.itemsData.getDataSet();e&&this.options.onRemove(e,function(e){e&&i.remove(t)})},s.prototype._getType=function(t){return t.type||this.options.type||(t.end?"range":"box")},s.prototype._getGroupId=function(t){var e=this._getType(t);return"background"==e&&void 0==t.group?g:this.groupsData?t.group:f},s.prototype._onUpdate=function(t){var e=this;t.forEach(function(t){var i=e.itemsData.get(t,e.itemOptions),o=e.items[t],n=e._getType(i),r=s.types[n];if(o&&(r&&o instanceof r?e._updateItem(o,i):(e._removeItem(o),o=null)),!o){if(!r)throw new TypeError("rangeoverflow"==n?'Item type "rangeoverflow" is deprecated. Use css styling instead: .vis.timeline .item.range .content {overflow: visible;}':'Unknown item type "'+n+'"');o=new r(i,e.conversion,e.options),o.id=t,e._addItem(o)}}),this._order(),this.stackDirty=!0,this.body.emitter.emit("change",{queue:!0})},s.prototype._onAdd=s.prototype._onUpdate,s.prototype._onRemove=function(t){var e=0,i=this;t.forEach(function(t){var s=i.items[t];s&&(e++,i._removeItem(s))}),e&&(this._order(),this.stackDirty=!0,this.body.emitter.emit("change",{queue:!0}))},s.prototype._order=function(){n.forEach(this.groups,function(t){t.order()})},s.prototype._onUpdateGroups=function(t){this._onAddGroups(t)},s.prototype._onAddGroups=function(t){var e=this;t.forEach(function(t){var i=e.groupsData.get(t),s=e.groups[t];if(s)s.setData(i);else{if(t==f||t==g)throw new Error("Illegal group id. "+t+" is a reserved id.");var o=Object.create(e.options);n.extend(o,{height:null}),s=new d(t,i,e),e.groups[t]=s;for(var r in e.items)if(e.items.hasOwnProperty(r)){var a=e.items[r];a.data.group==t&&s.add(a)}s.order(),s.show()}}),this.body.emitter.emit("change",{queue:!0})},s.prototype._onRemoveGroups=function(t){var e=this.groups;t.forEach(function(t){var i=e[t];i&&(i.hide(),delete e[t])}),this.markDirty(),this.body.emitter.emit("change",{queue:!0})},s.prototype._orderGroups=function(){if(this.groupsData){var t=this.groupsData.getIds({order:this.options.groupOrder}),e=!n.equalArray(t,this.groupIds);if(e){var i=this.groups;t.forEach(function(t){i[t].hide()}),t.forEach(function(t){i[t].show()}),this.groupIds=t}return e}return!1},s.prototype._addItem=function(t){this.items[t.id]=t;var e=this._getGroupId(t.data),i=this.groups[e];i&&i.add(t)},s.prototype._updateItem=function(t,e){var i=t.data.group;if(t.setData(e),i!=t.data.group){var s=this.groups[i];s&&s.remove(t);var o=this._getGroupId(t.data),n=this.groups[o];n&&n.add(t)}},s.prototype._removeItem=function(t){t.hide(),delete this.items[t.id];var e=this.selection.indexOf(t.id);-1!=e&&this.selection.splice(e,1),t.parent&&t.parent.remove(t)},s.prototype._constructByEndArray=function(t){for(var e=[],i=0;i0||o.length>0)&&this.body.emitter.emit("select",{items:a})}},s.prototype._onAddItem=function(t){if(this.options.selectable&&this.options.editable.add){var e=this,i=this.body.util.snap||null,o=s.itemFromTarget(t);if(o){var r=e.itemsData.get(o.id);this.options.onUpdate(r,function(t){t&&e.itemsData.getDataSet().update(t)})}else{var a=n.getAbsoluteLeft(this.dom.frame),h=t.center.x-a,d=this.body.util.toTime(h),l={start:i?i(d):d,content:"new item"};if("range"===this.options.type){var c=this.body.util.toTime(h+this.props.width/5);l.end=i?i(c):c}l[this.itemsData._fieldId]=n.randomUUID();var p=s.groupFromTarget(t);p&&(l.group=p.groupId),this.options.onAdd(l,function(t){t&&e.itemsData.getDataSet().add(t)})}}},s.prototype._onMultiSelectItem=function(t){if(this.options.selectable){var e,i=s.itemFromTarget(t);if(i){e=this.getSelection();var o=t.srcEvent&&t.srcEvent.shiftKey||!1;if(o){e.push(i.id);var n=s._getItemRange(this.itemsData.get(e,this.itemOptions));e=[];for(var r in this.items)if(this.items.hasOwnProperty(r)){var a=this.items[r],h=a.data.start,d=void 0!==a.data.end?a.data.end:h;h>=n.min&&d<=n.max&&e.push(a.id)}}else{var l=e.indexOf(i.id);-1==l?e.push(i.id):e.splice(l,1)}this.setSelection(e),this.body.emitter.emit("select",{items:this.getSelection()})}}},s._getItemRange=function(t){var e=null,i=null;return t.forEach(function(t){(null==i||t.starte)&&(e=t.end):(null==e||t.start>e)&&(e=t.start)}),{min:i,max:e}},s.itemFromTarget=function(t){for(var e=t.target;e;){if(e.hasOwnProperty("timeline-item"))return e["timeline-item"];e=e.parentNode}return null},s.groupFromTarget=function(t){for(var e=t.target;e;){if(e.hasOwnProperty("timeline-group"))return e["timeline-group"];e=e.parentNode}return null},s.itemSetFromTarget=function(t){for(var e=t.target;e;){if(e.hasOwnProperty("timeline-itemset"))return e["timeline-itemset"];e=e.parentNode}return null},t.exports=s},function(t,e,i){function s(t,e,i,s){this.body=t,this.defaultOptions={enabled:!0,icons:!0,iconSize:20,iconSpacing:6,left:{visible:!0,position:"top-left"},right:{visible:!0,position:"top-left"}},this.side=i,this.options=o.extend({},this.defaultOptions),this.linegraphOptions=s,this.svgElements={},this.dom={},this.groups={},this.amountOfGroups=0,this._create(),this.setOptions(e)}var o=i(1),n=i(2),r=i(20);s.prototype=new r,s.prototype.clear=function(){this.groups={},this.amountOfGroups=0},s.prototype.addGroup=function(t,e){this.groups.hasOwnProperty(t)||(this.groups[t]=e),this.amountOfGroups+=1},s.prototype.updateGroup=function(t,e){this.groups[t]=e},s.prototype.removeGroup=function(t){this.groups.hasOwnProperty(t)&&(delete this.groups[t],this.amountOfGroups-=1)},s.prototype._create=function(){this.dom.frame=document.createElement("div"),this.dom.frame.className="legend",this.dom.frame.style.position="absolute",this.dom.frame.style.top="10px",this.dom.frame.style.display="block",this.dom.textArea=document.createElement("div"),this.dom.textArea.className="legendText",this.dom.textArea.style.position="relative",this.dom.textArea.style.top="0px",this.svg=document.createElementNS("http://www.w3.org/2000/svg","svg"),this.svg.style.position="absolute",this.svg.style.top="0px",this.svg.style.width=this.options.iconSize+5+"px",this.svg.style.height="100%",this.dom.frame.appendChild(this.svg),this.dom.frame.appendChild(this.dom.textArea)},s.prototype.hide=function(){this.dom.frame.parentNode&&this.dom.frame.parentNode.removeChild(this.dom.frame)},s.prototype.show=function(){this.dom.frame.parentNode||this.body.dom.center.appendChild(this.dom.frame)},s.prototype.setOptions=function(t){var e=["enabled","orientation","icons","left","right"];o.selectiveDeepExtend(e,this.options,t)},s.prototype.redraw=function(){var t=0;for(var e in this.groups)this.groups.hasOwnProperty(e)&&(1!=this.groups[e].visible||void 0!==this.linegraphOptions.visibility[e]&&1!=this.linegraphOptions.visibility[e]||t++);if(0==this.options[this.side].visible||0==this.amountOfGroups||0==this.options.enabled||0==t)this.hide();else{if(this.show(),"top-left"==this.options[this.side].position||"bottom-left"==this.options[this.side].position?(this.dom.frame.style.left="4px",this.dom.frame.style.textAlign="left",this.dom.textArea.style.textAlign="left",this.dom.textArea.style.left=this.options.iconSize+15+"px",this.dom.textArea.style.right="",this.svg.style.left="0px",this.svg.style.right=""):(this.dom.frame.style.right="4px",this.dom.frame.style.textAlign="right",this.dom.textArea.style.textAlign="right",this.dom.textArea.style.right=this.options.iconSize+15+"px",this.dom.textArea.style.left="",this.svg.style.right="0px",this.svg.style.left=""),"top-left"==this.options[this.side].position||"top-right"==this.options[this.side].position)this.dom.frame.style.top=4-Number(this.body.dom.center.style.top.replace("px",""))+"px",this.dom.frame.style.bottom="";else{var i=this.body.domProps.center.height-this.body.domProps.centerContainer.height;this.dom.frame.style.bottom=4+i+Number(this.body.dom.center.style.top.replace("px",""))+"px",this.dom.frame.style.top=""}0==this.options.icons?(this.dom.frame.style.width=this.dom.textArea.offsetWidth+10+"px",this.dom.textArea.style.right="",this.dom.textArea.style.left="",this.svg.style.width="0px"):(this.dom.frame.style.width=this.options.iconSize+15+this.dom.textArea.offsetWidth+10+"px",this.drawLegendIcons());var s="";for(var e in this.groups)this.groups.hasOwnProperty(e)&&(1!=this.groups[e].visible||void 0!==this.linegraphOptions.visibility[e]&&1!=this.linegraphOptions.visibility[e]||(s+=this.groups[e].content+"
"));this.dom.textArea.innerHTML=s,this.dom.textArea.style.lineHeight=.75*this.options.iconSize+this.options.iconSpacing+"px"}},s.prototype.drawLegendIcons=function(){if(this.dom.frame.parentNode){n.prepareElements(this.svgElements);var t=window.getComputedStyle(this.dom.frame).paddingTop,e=Number(t.replace("px","")),i=e,s=this.options.iconSize,o=.75*this.options.iconSize,r=e+.5*o+3;this.svg.style.width=s+5+e+"px";for(var a in this.groups)this.groups.hasOwnProperty(a)&&(1!=this.groups[a].visible||void 0!==this.linegraphOptions.visibility[a]&&1!=this.linegraphOptions.visibility[a]||(this.groups[a].drawIcon(i,r,this.svgElements,this.svg,s,o),r+=o+this.options.iconSpacing));n.cleanupElements(this.svgElements)}},t.exports=s},function(t,e,i){function s(t,e){this.id=o.randomUUID(),this.body=t,this.defaultOptions={yAxisOrientation:"left",defaultGroup:"default",sort:!0,sampling:!0,graphHeight:"400px",shaded:{enabled:!1,orientation:"bottom"},style:"line",barChart:{width:50,handleOverlap:"overlap",align:"center"},catmullRom:{enabled:!0,parametrization:"centripetal",alpha:.5},drawPoints:{enabled:!0,size:6,style:"square"},dataAxis:{showMinorLabels:!0,showMajorLabels:!0,icons:!1,width:"40px",visible:!0,alignZeros:!0,customRange:{left:{min:void 0,max:void 0},right:{min:void 0,max:void 0}}},legend:{enabled:!1,icons:!0,left:{visible:!0,position:"top-left"},right:{visible:!0,position:"top-right"}},groups:{visibility:{}}},this.options=o.extend({},this.defaultOptions),this.dom={},this.props={},this.hammer=null,this.groups={},this.abortedGraphUpdate=!1,this.updateSVGheight=!1,this.updateSVGheightOnResize=!1;var i=this;this.itemsData=null,this.groupsData=null,this.itemListeners={add:function(t,e){i._onAdd(e.items)},update:function(t,e){i._onUpdate(e.items)},remove:function(t,e){i._onRemove(e.items)}},this.groupListeners={add:function(t,e){i._onAddGroups(e.items)},update:function(t,e){i._onUpdateGroups(e.items)},remove:function(t,e){i._onRemoveGroups(e.items)}},this.items={},this.selection=[],this.lastStart=this.body.range.start,this.touchParams={},this.svgElements={},this.setOptions(e),this.groupsUsingDefaultStyles=[0],this.COUNTER=0,this.body.emitter.on("rangechanged",function(){i.lastStart=i.body.range.start,i.svg.style.left=o.option.asSize(-i.props.width),i.redraw.call(i,!0)}),this._create(),this.framework={svg:this.svg,svgElements:this.svgElements,options:this.options,groups:this.groups},this.body.emitter.emit("change")}var o=i(1),n=i(2),r=i(3),a=i(4),h=i(20),d=i(23),l=i(24),c=i(28),p=i(52),u="__ungrouped__";s.prototype=new h,s.prototype._create=function(){var t=document.createElement("div");t.className="LineGraph",this.dom.frame=t,this.svg=document.createElementNS("http://www.w3.org/2000/svg","svg"),this.svg.style.position="relative",this.svg.style.height=(""+this.options.graphHeight).replace("px","")+"px",this.svg.style.display="block",t.appendChild(this.svg),this.options.dataAxis.orientation="left",this.yAxisLeft=new d(this.body,this.options.dataAxis,this.svg,this.options.groups),this.options.dataAxis.orientation="right",this.yAxisRight=new d(this.body,this.options.dataAxis,this.svg,this.options.groups),delete this.options.dataAxis.orientation,this.legendLeft=new c(this.body,this.options.legend,"left",this.options.groups),this.legendRight=new c(this.body,this.options.legend,"right",this.options.groups),this.show()},s.prototype.setOptions=function(t){if(t){var e=["sampling","defaultGroup","height","graphHeight","yAxisOrientation","style","barChart","dataAxis","sort","groups"];void 0===t.graphHeight&&void 0!==t.height&&void 0!==this.body.domProps.centerContainer.height?(this.updateSVGheight=!0,this.updateSVGheightOnResize=!0):void 0!==this.body.domProps.centerContainer.height&&void 0!==t.graphHeight&&parseInt((t.graphHeight+"").replace("px",""))0){var d=this.body.util.toGlobalTime(-this.body.domProps.root.width),l=this.body.util.toGlobalTime(2*this.body.domProps.root.width),c={};for(this._getRelevantData(a,c,d,l),this._applySampling(a,c),e=0;eu&&console.log("WARNING: there may be an infinite loop in the _updateGraph emitter cycle."),this.COUNTER=0,this.abortedGraphUpdate=!1,e=0;e0)for(r=0;rs){d.push(h);break}d.push(h)}}else for(a=0;ai&&h.x0)for(var s=0;s0){var n=1,r=o.length,a=this.body.util.toGlobalScreen(o[o.length-1].x)-this.body.util.toGlobalScreen(o[0].x),h=r/a;n=Math.min(Math.ceil(.2*r),Math.max(1,Math.round(h)));for(var d=[],l=0;r>l;l+=n)d.push(o[l]);e[t[s]]=d}}},s.prototype._getYRanges=function(t,e,i){var s,o,n,r,a=[],h=[];if(t.length>0){for(n=0;n0&&(o=this.groups[t[n]],"stack"==r.barChart.handleOverlap&&"bar"==r.style?"left"==r.yAxisOrientation?a=a.concat(o.getYRange(s)):h=h.concat(o.getYRange(s)):i[t[n]]=o.getYRange(s,t[n]));p.getStackedBarYRange(a,i,t,"__barchartLeft","left"),p.getStackedBarYRange(h,i,t,"__barchartRight","right")}},s.prototype._updateYAxis=function(t,e){var i,s,o=!1,n=!1,r=!1,a=1e9,h=1e9,d=-1e9,l=-1e9;if(t.length>0){for(var c=0;ci?i:a,d=s>d?s:d):(r=!0,h=h>i?i:h,l=s>l?s:l));1==n&&this.yAxisLeft.setRange(a,d),1==r&&this.yAxisRight.setRange(h,l)}return o=this._toggleAxisVisiblity(n,this.yAxisLeft)||o,o=this._toggleAxisVisiblity(r,this.yAxisRight)||o,1==r&&1==n?(this.yAxisLeft.drawIcons=!0,this.yAxisRight.drawIcons=!0):(this.yAxisLeft.drawIcons=!1,this.yAxisRight.drawIcons=!1),this.yAxisRight.master=!n,0==this.yAxisRight.master?(this.yAxisLeft.lineOffset=1==r?this.yAxisRight.width:0,o=this.yAxisLeft.redraw()||o,this.yAxisRight.stepPixelsForced=this.yAxisLeft.stepPixels,this.yAxisRight.zeroCrossing=this.yAxisLeft.zeroCrossing,o=this.yAxisRight.redraw()||o):o=this.yAxisRight.redraw()||o,-1!=t.indexOf("__barchartLeft")&&t.splice(t.indexOf("__barchartLeft"),1),-1!=t.indexOf("__barchartRight")&&t.splice(t.indexOf("__barchartRight"),1),o},s.prototype._toggleAxisVisiblity=function(t,e){var i=!1;return 0==t?e.dom.frame.parentNode&&0==e.hidden&&(e.hide(),i=!0):e.dom.frame.parentNode||1!=e.hidden||(e.show(),i=!0),i},s.prototype._convertXcoordinates=function(t){for(var e,i,s=[],o=this.body.util.toScreen,n=0;ny;)y++,l=h.getCurrent(),c=h.isMajor(),u=h.getClassName(),f=m,m=this.body.util.toScreen(l),g=m-f,p&&(p.style.width=g+"px"),this.options.showMinorLabels&&this._repaintMinorText(m,h.getLabelMinor(),t,u),c&&this.options.showMajorLabels?(m>0&&(void 0==v&&(v=m),this._repaintMajorText(m,h.getLabelMajor(),t,u)),p=this._repaintMajorLine(m,t,u)):p=this._repaintMinorLine(m,t,u),h.next();if(this.options.showMajorLabels){var b=this.body.util.toTime(0),_=h.getLabelMajor(b),x=_.length*(this.props.majorCharWidth||10)+10;(void 0==v||v>x)&&this._repaintMajorText(0,_,t,u)}o.forEach(this.dom.redundant,function(t){for(;t.length;){var e=t.pop();e&&e.parentNode&&e.parentNode.removeChild(e)}})},s.prototype._repaintMinorText=function(t,e,i,s){var o=this.dom.redundant.minorTexts.shift();if(!o){var n=document.createTextNode("");o=document.createElement("div"),o.appendChild(n),this.dom.foreground.appendChild(o)}this.dom.minorTexts.push(o),o.childNodes[0].nodeValue=e,o.style.top="top"==i?this.props.majorLabelHeight+"px":"0",o.style.left=t+"px",o.className="text minor "+s},s.prototype._repaintMajorText=function(t,e,i,s){var o=this.dom.redundant.majorTexts.shift();if(!o){var n=document.createTextNode(e);o=document.createElement("div"),o.appendChild(n),this.dom.foreground.appendChild(o)}this.dom.majorTexts.push(o),o.childNodes[0].nodeValue=e,o.className="text major "+s,o.style.top="top"==i?"0":this.props.minorLabelHeight+"px",o.style.left=t+"px"},s.prototype._repaintMinorLine=function(t,e,i){var s=this.dom.redundant.lines.shift();s||(s=document.createElement("div"),this.dom.background.appendChild(s)),this.dom.lines.push(s);var o=this.props;return s.style.top="top"==e?o.majorLabelHeight+"px":this.body.domProps.top.height+"px",s.style.height=o.minorLineHeight+"px",s.style.left=t-o.minorLineWidth/2+"px",s.className="grid vertical minor "+i,s},s.prototype._repaintMajorLine=function(t,e,i){var s=this.dom.redundant.lines.shift();s||(s=document.createElement("div"),this.dom.background.appendChild(s)),this.dom.lines.push(s);var o=this.props;return s.style.top="top"==e?"0":this.body.domProps.top.height+"px",s.style.left=t-o.majorLineWidth/2+"px",s.style.height=o.majorLineHeight+"px",s.className="grid vertical major "+i,s},s.prototype._calculateCharSize=function(){this.dom.measureCharMinor||(this.dom.measureCharMinor=document.createElement("DIV"),this.dom.measureCharMinor.className="text minor measure",this.dom.measureCharMinor.style.position="absolute",this.dom.measureCharMinor.appendChild(document.createTextNode("0")),this.dom.foreground.appendChild(this.dom.measureCharMinor)),this.props.minorCharHeight=this.dom.measureCharMinor.clientHeight,this.props.minorCharWidth=this.dom.measureCharMinor.clientWidth,this.dom.measureCharMajor||(this.dom.measureCharMajor=document.createElement("DIV"),this.dom.measureCharMajor.className="text major measure",this.dom.measureCharMajor.style.position="absolute",this.dom.measureCharMajor.appendChild(document.createTextNode("0")),this.dom.foreground.appendChild(this.dom.measureCharMajor)),this.props.majorCharHeight=this.dom.measureCharMajor.clientHeight,this.props.majorCharWidth=this.dom.measureCharMajor.clientWidth},s.prototype.snap=function(t){return this.step.snap(t)},t.exports=s},function(t,e,i){function s(t,e,i){this.id=null,this.parent=null,this.data=t,this.dom=null,this.conversion=e||{},this.options=i||{},this.selected=!1,this.displayed=!1,this.dirty=!0,this.top=null,this.left=null,this.width=null,this.height=null}var o=i(45),n=i(1);s.prototype.stack=!0,s.prototype.select=function(){this.selected=!0,this.dirty=!0,this.displayed&&this.redraw()},s.prototype.unselect=function(){this.selected=!1,this.dirty=!0,this.displayed&&this.redraw()},s.prototype.setData=function(t){this.data=t,this.dirty=!0,this.displayed&&this.redraw()},s.prototype.setParent=function(t){this.displayed?(this.hide(),this.parent=t,this.parent&&this.show()):this.parent=t},s.prototype.isVisible=function(){return!1},s.prototype.show=function(){return!1},s.prototype.hide=function(){return!1},s.prototype.redraw=function(){},s.prototype.repositionX=function(){},s.prototype.repositionY=function(){},s.prototype._repaintDeleteButton=function(t){if(this.selected&&this.options.editable.remove&&!this.dom.deleteButton){var e=this,i=document.createElement("div");i.className="delete",i.title="Delete this item",new o(i).on("tap",function(t){e.parent.removeFromDataSet(e),t.stopPropagation(),t.preventDefault()}),t.appendChild(i),this.dom.deleteButton=i}else!this.selected&&this.dom.deleteButton&&(this.dom.deleteButton.parentNode&&this.dom.deleteButton.parentNode.removeChild(this.dom.deleteButton),this.dom.deleteButton=null)},s.prototype._updateContents=function(t){var e;if(this.options.template){var i=this.parent.itemSet.itemsData.get(this.id);e=this.options.template(i)}else e=this.data.content;if(e!==this.content){if(e instanceof Element)t.innerHTML="",t.appendChild(e);else if(void 0!=e)t.innerHTML=e;else if("background"!=this.data.type||void 0!==this.data.content)throw new Error('Property "content" missing in item '+this.id);this.content=e}},s.prototype._updateTitle=function(t){null!=this.data.title?t.title=this.data.title||"":t.removeAttribute("title")},s.prototype._updateDataAttributes=function(t){if(this.options.dataAttributes&&this.options.dataAttributes.length>0){var e=[];if(Array.isArray(this.options.dataAttributes))e=this.options.dataAttributes;else{if("all"!=this.options.dataAttributes)return;e=Object.keys(this.data)}for(var i=0;it.start},s.prototype.redraw=function(){var t=this.dom;if(t||(this.dom={},t=this.dom,t.box=document.createElement("div"),t.content=document.createElement("div"),t.content.className="content",t.box.appendChild(t.content),this.dirty=!0),!this.parent)throw new Error("Cannot redraw item: no parent attached");if(!t.box.parentNode){var e=this.parent.dom.background;if(!e)throw new Error("Cannot redraw item: parent has no background container element");e.appendChild(t.box)}if(this.displayed=!0,this.dirty){this._updateContents(this.dom.content),this._updateTitle(this.dom.content),this._updateDataAttributes(this.dom.content),this._updateStyle(this.dom.box);var i=(this.data.className?" "+this.data.className:"")+(this.selected?" selected":"");t.box.className=this.baseClassName+i,this.overflow="hidden"!==window.getComputedStyle(t.content).overflow,this.props.content.width=this.dom.content.offsetWidth,this.height=0,this.dirty=!1}},s.prototype.show=r.prototype.show,s.prototype.hide=r.prototype.hide,s.prototype.repositionX=r.prototype.repositionX,s.prototype.repositionY=function(t){var e="top"===this.options.orientation;this.dom.content.style.top=e?"":"0",this.dom.content.style.bottom=e?"0":"";var i;if(void 0!==this.data.subgroup){var s=this.data.subgroup,o=this.parent.subgroups,r=o[s].index;if(1==e){i=this.parent.subgroups[s].height+t.item.vertical,i+=0==r?t.axis-.5*t.item.vertical:0;var a=this.parent.top;for(var h in o)o.hasOwnProperty(h)&&1==o[h].visible&&o[h].indexr&&(a+=o[h].height+t.item.vertical);i=this.parent.subgroups[s].height+t.item.vertical,this.dom.box.style.top=a+"px",this.dom.box.style.bottom=""}}else this.parent instanceof n?(i=Math.max(this.parent.height,this.parent.itemSet.body.domProps.center.height,this.parent.itemSet.body.domProps.centerContainer.height),this.dom.box.style.top=e?"0":"",this.dom.box.style.bottom=e?"":"0"):(i=this.parent.height,this.dom.box.style.top=this.parent.top+"px",this.dom.box.style.bottom="");this.dom.box.style.height=i+"px"},t.exports=s},function(t,e,i){function s(t,e,i){if(this.props={dot:{width:0,height:0},line:{width:0,height:0}},t&&void 0==t.start)throw new Error('Property "start" missing in item '+t);o.call(this,t,e,i)}{var o=i(31);i(1)}s.prototype=new o(null,null,null),s.prototype.isVisible=function(t){var e=(t.end-t.start)/4;return this.data.start>t.start-e&&this.data.startt.start-e&&this.data.startt.start},s.prototype.redraw=function(){var t=this.dom;if(t||(this.dom={},t=this.dom,t.box=document.createElement("div"),t.content=document.createElement("div"),t.content.className="content",t.box.appendChild(t.content),t.box["timeline-item"]=this,this.dirty=!0),!this.parent)throw new Error("Cannot redraw item: no parent attached");if(!t.box.parentNode){var e=this.parent.dom.foreground;if(!e)throw new Error("Cannot redraw item: parent has no foreground container element");e.appendChild(t.box)}if(this.displayed=!0,this.dirty){this._updateContents(this.dom.content),this._updateTitle(this.dom.box),this._updateDataAttributes(this.dom.box),this._updateStyle(this.dom.box);var i=(this.data.className?" "+this.data.className:"")+(this.selected?" selected":"");t.box.className=this.baseClassName+i,this.overflow="hidden"!==window.getComputedStyle(t.content).overflow,this.dom.content.style.maxWidth="none",this.props.content.width=this.dom.content.offsetWidth,this.height=this.dom.box.offsetHeight,this.dom.content.style.maxWidth="",this.dirty=!1}this._repaintDeleteButton(t.box),this._repaintDragLeft(),this._repaintDragRight()},s.prototype.show=function(){this.displayed||this.redraw()},s.prototype.hide=function(){if(this.displayed){var t=this.dom.box;t.parentNode&&t.parentNode.removeChild(t),this.top=null,this.left=null,this.displayed=!1}},s.prototype.repositionX=function(){var t,e,i=this.parent.width,s=this.conversion.toScreen(this.data.start),o=this.conversion.toScreen(this.data.end);-i>s&&(s=-i),o>2*i&&(o=2*i);var n=Math.max(o-s,1);switch(this.overflow?(this.left=s,this.width=n+this.props.content.width,e=this.props.content.width):(this.left=s,this.width=n,e=Math.min(o-s-2*this.options.padding,this.props.content.width)),this.dom.box.style.left=this.left+"px",this.dom.box.style.width=n+"px",this.options.align){case"left":this.dom.content.style.left="0";break;case"right":this.dom.content.style.left=Math.max(n-e-2*this.options.padding,0)+"px";break;case"center":this.dom.content.style.left=Math.max((n-e-2*this.options.padding)/2,0)+"px";break;default:t=this.overflow?o>0?Math.max(-s,0):-e:0>s?Math.min(-s,o-s-e-2*this.options.padding):0,this.dom.content.style.left=t+"px"}},s.prototype.repositionY=function(){var t=this.options.orientation,e=this.dom.box;e.style.top="top"==t?this.top+"px":this.parent.height-this.top-this.height+"px"},s.prototype._repaintDragLeft=function(){if(this.selected&&this.options.editable.updateTime&&!this.dom.dragLeft){var t=document.createElement("div");t.className="drag-left",t.dragLeftItem=this,this.dom.box.appendChild(t),this.dom.dragLeft=t}else!this.selected&&this.dom.dragLeft&&(this.dom.dragLeft.parentNode&&this.dom.dragLeft.parentNode.removeChild(this.dom.dragLeft),this.dom.dragLeft=null)},s.prototype._repaintDragRight=function(){if(this.selected&&this.options.editable.updateTime&&!this.dom.dragRight){var t=document.createElement("div");t.className="drag-right",t.dragRightItem=this,this.dom.box.appendChild(t),this.dom.dragRight=t}else!this.selected&&this.dom.dragRight&&(this.dom.dragRight.parentNode&&this.dom.dragRight.parentNode.removeChild(this.dom.dragRight),this.dom.dragRight=null)},t.exports=s},function(t,e,i){function s(t,e,i){if(!(this instanceof s))throw new SyntaxError("Constructor must be called with the new operator");this._determineBrowserMethod(),this._initializeMixinLoaders(),this.containerElement=t,this.renderRefreshRate=60,this.renderTimestep=1e3/this.renderRefreshRate,this.renderTime=0,this.physicsTime=0,this.runDoubleSpeed=!1,this.physicsDiscreteStepsize=.5,this.initializing=!0,this.triggerFunctions={add:null,edit:null,editEdge:null,connect:null,del:null},this.defaultOptions={nodes:{mass:1,radiusMin:10,radiusMax:30,radius:10,shape:"ellipse",image:void 0,widthMin:16,widthMax:64,fontColor:"black",fontSize:14,fontFace:"verdana",fontFill:void 0,fontStrokeWidth:0,fontStrokeColor:"white",level:-1,color:{border:"#2B7CE9",background:"#97C2FC",highlight:{border:"#2B7CE9",background:"#D2E5FF"},hover:{border:"#2B7CE9",background:"#D2E5FF"}},group:void 0,borderWidth:1,borderWidthSelected:void 0},edges:{widthMin:1,widthMax:15,width:1,widthSelectionMultiplier:2,hoverWidth:1.5,style:"line",color:{color:"#848484",highlight:"#848484",hover:"#848484"},fontColor:"#343434",fontSize:14,fontFace:"arial",fontFill:"white",fontStrokeWidth:0,fontStrokeColor:"white",labelAlignment:"horizontal",arrowScaleFactor:1,dash:{length:10,gap:5,altLength:void 0},inheritColor:"from"},configurePhysics:!1,physics:{barnesHut:{enabled:!0,thetaInverted:2,gravitationalConstant:-2e3,centralGravity:.3,springLength:95,springConstant:.04,damping:.09},repulsion:{centralGravity:0,springLength:200,springConstant:.05,nodeDistance:100,damping:.09},hierarchicalRepulsion:{enabled:!1,centralGravity:0,springLength:100,springConstant:.01,nodeDistance:150,damping:.09},damping:null,centralGravity:null,springLength:null,springConstant:null},clustering:{enabled:!1,initialMaxNodes:100,clusterThreshold:500,reduceToNodes:300,chainThreshold:.4,clusterEdgeThreshold:20,sectorThreshold:100,screenSizeThreshold:.2,fontSizeMultiplier:4,maxFontSize:1e3,forceAmplification:.1,distanceAmplification:.1,edgeGrowth:20,nodeScaling:{width:1,height:1,radius:1},maxNodeSizeIncrements:600,activeAreaBoxSize:80,clusterLevelDifference:2},navigation:{enabled:!1},keyboard:{enabled:!1,speed:{x:10,y:10,zoom:.02}},dataManipulation:{enabled:!1,initiallyVisible:!1},hierarchicalLayout:{enabled:!1,levelSeparation:150,nodeSpacing:100,direction:"UD",layout:"hubsize"},freezeForStabilization:!1,smoothCurves:{enabled:!0,dynamic:!0,type:"continuous",roundness:.5},maxVelocity:30,minVelocity:.1,stabilize:!0,stabilizationIterations:1e3,zoomExtentOnStabilize:!0,locale:"en",locales:_,tooltip:{delay:300,fontColor:"black",fontSize:14,fontFace:"verdana",color:{border:"#666",background:"#FFFFC6"}},dragNetwork:!0,dragNodes:!0,zoomable:!0,hover:!1,hideEdgesOnDrag:!1,hideNodesOnDrag:!1,width:"100%",height:"100%",selectable:!0},this.constants=a.extend({},this.defaultOptions),this.pixelRatio=1,this.hoverObj={nodes:{},edges:{}},this.controlNodesActive=!1,this.navigationHammers={existing:[],_new:[]},this.animationSpeed=1/this.renderRefreshRate,this.animationEasingFunction="easeInOutQuint",this.easingTime=0,this.sourceScale=0,this.targetScale=0,this.sourceTranslation=0,this.targetTranslation=0,this.lockedOnNodeId=null,this.lockedOnNodeOffset=null,this.touchTime=0;var o=this;this.groups=new u,this.images=new m,this.images.setOnloadCallback(function(){o._redraw()}),this.xIncrement=0,this.yIncrement=0,this.zoomIncrement=0,this._loadPhysicsSystem(),this._create(),this._loadSectorSystem(),this._loadClusterSystem(),this._loadSelectionSystem(),this._loadHierarchySystem(),this._setTranslation(this.frame.clientWidth/2,this.frame.clientHeight/2),this._setScale(1),this.setOptions(i),this.freezeSimulation=!1,this.cachedFunctions={},this.startedStabilization=!1,this.stabilized=!1,this.stabilizationIterations=null,this.draggingNodes=!1,this.calculationNodes={},this.calculationNodeIndices=[],this.nodeIndices=[],this.nodes={},this.edges={},this.canvasTopLeft={x:0,y:0},this.canvasBottomRight={x:0,y:0},this.pointerPosition={x:0,y:0},this.areaCenter={},this.scale=1,this.previousScale=this.scale,this.nodesData=null,this.edgesData=null,this.nodesListeners={add:function(t,e){o._addNodes(e.items),o.start()},update:function(t,e){o._updateNodes(e.items,e.data),o.start()},remove:function(t,e){o._removeNodes(e.items),o.start()}},this.edgesListeners={add:function(t,e){o._addEdges(e.items),o.start()},update:function(t,e){o._updateEdges(e.items),o.start()},remove:function(t,e){o._removeEdges(e.items),o.start()}},this.moving=!0,this.timer=void 0,this.setData(e,this.constants.clustering.enabled||this.constants.hierarchicalLayout.enabled),this.initializing=!1,1==this.constants.hierarchicalLayout.enabled?this._setupHierarchicalLayout():0==this.constants.stabilize&&this.zoomExtent(void 0,!0,this.constants.clustering.enabled),this.constants.clustering.enabled&&this.startWithClustering()}var o=i(56),n=i(45),r=i(58),a=i(1),h=i(47),d=i(3),l=i(4),c=i(42),p=i(43),u=i(38),m=i(39),f=i(40),g=i(37),v=i(41),y=i(54),b=i(55),_=i(49);i(50),o(s.prototype),s.prototype._determineBrowserMethod=function(){var t=navigator.userAgent.toLowerCase();this.requiresTimeout=!1,-1!=t.indexOf("msie 9.0")?this.requiresTimeout=!0:-1!=t.indexOf("safari")&&t.indexOf("chrome")<=-1&&(this.requiresTimeout=!0)},s.prototype._getScriptPath=function(){for(var t=document.getElementsByTagName("script"),e=0;et.boundingBox.left&&(s=t.boundingBox.left),ot.boundingBox.bottom&&(e=t.boundingBox.bottom),i=this.constants.clustering.initialMaxNodes?49.07548/(n+142.05338)+91444e-8:12.662/(n+7.4147)+.0964822:1==this.constants.clustering.enabled&&n>=this.constants.clustering.initialMaxNodes?77.5271985/(n+187.266146)+476710517e-13:30.5062972/(n+19.93597763)+.08413486;var r=Math.min(this.frame.canvas.clientWidth/600,this.frame.canvas.clientHeight/600);s*=r}else{var a=1.1*Math.abs(o.maxX-o.minX),h=1.1*Math.abs(o.maxY-o.minY),d=this.frame.canvas.clientWidth/a,l=this.frame.canvas.clientHeight/h;s=l>=d?d:l}s>1&&(s=1);var c=this._findCenter(o);if(0==i){var p={position:c,scale:s,animation:t};this.moveTo(p),this.moving=!0,this.start()}else c.x*=s,c.y*=s,c.x-=.5*this.frame.canvas.clientWidth,c.y-=.5*this.frame.canvas.clientHeight,this._setScale(s),this._setTranslation(-c.x,-c.y)},s.prototype._updateNodeIndexList=function(){this._clearNodeIndexList();for(var t in this.nodes)this.nodes.hasOwnProperty(t)&&this.nodeIndices.push(t)},s.prototype.setData=function(t,e){if(void 0===e&&(e=!1),this.initializing=!0,t&&t.dot&&(t.nodes||t.edges))throw new SyntaxError('Data must contain either parameter "dot" or parameter pair "nodes" and "edges", but not both.');if(1==this.constants.dataManipulation.enabled&&this._createManipulatorBar(),this.setOptions(t&&t.options),t&&t.dot){if(t&&t.dot){var i=c.DOTToGraph(t.dot);return void this.setData(i)}}else if(t&&t.gephi){if(t&&t.gephi){var s=p.parseGephi(t.gephi);return void this.setData(s)}}else this._setNodes(t&&t.nodes),this._setEdges(t&&t.edges);this._putDataInSector(),0==e&&(1==this.constants.hierarchicalLayout.enabled?(this._resetLevels(),this._setupHierarchicalLayout()):this.constants.stabilize&&this._stabilize(),this.start()),this.initializing=!1},s.prototype.setOptions=function(t){if(t){var e,i=["nodes","edges","smoothCurves","hierarchicalLayout","clustering","navigation","keyboard","dataManipulation","onAdd","onEdit","onEditEdge","onConnect","onDelete","clickToUse"];if(a.selectiveNotDeepExtend(i,this.constants,t),a.selectiveNotDeepExtend(["color"],this.constants.nodes,t.nodes),a.selectiveNotDeepExtend(["color","length"],this.constants.edges,t.edges),t.physics&&(a.mergeOptions(this.constants.physics,t.physics,"barnesHut"),a.mergeOptions(this.constants.physics,t.physics,"repulsion"),t.physics.hierarchicalRepulsion)){this.constants.hierarchicalLayout.enabled=!0,this.constants.physics.hierarchicalRepulsion.enabled=!0,this.constants.physics.barnesHut.enabled=!1;for(e in t.physics.hierarchicalRepulsion)t.physics.hierarchicalRepulsion.hasOwnProperty(e)&&(this.constants.physics.hierarchicalRepulsion[e]=t.physics.hierarchicalRepulsion[e]) -}if(t.onAdd&&(this.triggerFunctions.add=t.onAdd),t.onEdit&&(this.triggerFunctions.edit=t.onEdit),t.onEditEdge&&(this.triggerFunctions.editEdge=t.onEditEdge),t.onConnect&&(this.triggerFunctions.connect=t.onConnect),t.onDelete&&(this.triggerFunctions.del=t.onDelete),a.mergeOptions(this.constants,t,"smoothCurves"),a.mergeOptions(this.constants,t,"hierarchicalLayout"),a.mergeOptions(this.constants,t,"clustering"),a.mergeOptions(this.constants,t,"navigation"),a.mergeOptions(this.constants,t,"keyboard"),a.mergeOptions(this.constants,t,"dataManipulation"),t.dataManipulation&&(this.editMode=this.constants.dataManipulation.initiallyVisible),t.edges&&(void 0!==t.edges.color&&(a.isString(t.edges.color)?(this.constants.edges.color={},this.constants.edges.color.color=t.edges.color,this.constants.edges.color.highlight=t.edges.color,this.constants.edges.color.hover=t.edges.color):(void 0!==t.edges.color.color&&(this.constants.edges.color.color=t.edges.color.color),void 0!==t.edges.color.highlight&&(this.constants.edges.color.highlight=t.edges.color.highlight),void 0!==t.edges.color.hover&&(this.constants.edges.color.hover=t.edges.color.hover)),this.constants.edges.inheritColor=!1),t.edges.fontColor||void 0!==t.edges.color&&(a.isString(t.edges.color)?this.constants.edges.fontColor=t.edges.color:void 0!==t.edges.color.color&&(this.constants.edges.fontColor=t.edges.color.color))),t.nodes&&t.nodes.color){var s=a.parseColor(t.nodes.color);this.constants.nodes.color.background=s.background,this.constants.nodes.color.border=s.border,this.constants.nodes.color.highlight.background=s.highlight.background,this.constants.nodes.color.highlight.border=s.highlight.border,this.constants.nodes.color.hover.background=s.hover.background,this.constants.nodes.color.hover.border=s.hover.border}if(t.groups)for(var o in t.groups)if(t.groups.hasOwnProperty(o)){var n=t.groups[o];this.groups.add(o,n)}if(t.tooltip){for(e in t.tooltip)t.tooltip.hasOwnProperty(e)&&(this.constants.tooltip[e]=t.tooltip[e]);t.tooltip.color&&(this.constants.tooltip.color=a.parseColor(t.tooltip.color))}if("clickToUse"in t&&(t.clickToUse?this.activator||(this.activator=new b(this.frame),this.activator.on("change",this._createKeyBinds.bind(this))):this.activator&&(this.activator.destroy(),delete this.activator)),t.labels)throw new Error('Option "labels" is deprecated. Use options "locale" and "locales" instead.');this._loadPhysicsSystem(),this._loadNavigationControls(),this._loadManipulationSystem(),this._configureSmoothCurves(),this._createKeyBinds(),this.setSize(this.constants.width,this.constants.height),this.moving=!0,this.start()}},s.prototype._create=function(){for(;this.containerElement.hasChildNodes();)this.containerElement.removeChild(this.containerElement.firstChild);if(this.frame=document.createElement("div"),this.frame.className="vis network-frame",this.frame.style.position="relative",this.frame.style.overflow="hidden",this.frame.canvas=document.createElement("canvas"),this.frame.canvas.style.position="relative",this.frame.appendChild(this.frame.canvas),this.frame.canvas.getContext){var t=this.frame.canvas.getContext("2d");this.pixelRatio=(window.devicePixelRatio||1)/(t.webkitBackingStorePixelRatio||t.mozBackingStorePixelRatio||t.msBackingStorePixelRatio||t.oBackingStorePixelRatio||t.backingStorePixelRatio||1),this.frame.canvas.getContext("2d").setTransform(this.pixelRatio,0,0,this.pixelRatio,0,0)}else{var e=document.createElement("DIV");e.style.color="red",e.style.fontWeight="bold",e.style.padding="10px",e.innerHTML="Error: your browser does not support HTML canvas",this.frame.canvas.appendChild(e)}var i=this;this.drag={},this.pinch={},this.hammer=new n(this.frame.canvas),this.hammer.get("pinch").set({enable:!0}),this.hammer.on("tap",i._onTap.bind(i)),this.hammer.on("doubletap",i._onDoubleTap.bind(i)),this.hammer.on("press",i._onHold.bind(i)),this.hammer.on("pinch",i._onPinch.bind(i)),h.onTouch(this.hammer,i._onTouch.bind(i)),this.hammer.on("panstart",i._onDragStart.bind(i)),this.hammer.on("panmove",i._onDrag.bind(i)),this.hammer.on("panend",i._onDragEnd.bind(i)),this.frame.canvas.addEventListener("mousemove",i._onMouseMoveTitle.bind(i)),this.frame.canvas.addEventListener("mousewheel",i._onMouseWheel.bind(i)),this.frame.canvas.addEventListener("DOMMouseScroll",i._onMouseWheel.bind(i)),this.containerElement.appendChild(this.frame)},s.prototype._createKeyBinds=function(){var t=this;void 0!==this.keycharm&&this.keycharm.destroy(),this.keycharm=r(),this.keycharm.reset(),this.constants.keyboard.enabled&&this.isActive()&&(this.keycharm.bind("up",this._moveUp.bind(t),"keydown"),this.keycharm.bind("up",this._yStopMoving.bind(t),"keyup"),this.keycharm.bind("down",this._moveDown.bind(t),"keydown"),this.keycharm.bind("down",this._yStopMoving.bind(t),"keyup"),this.keycharm.bind("left",this._moveLeft.bind(t),"keydown"),this.keycharm.bind("left",this._xStopMoving.bind(t),"keyup"),this.keycharm.bind("right",this._moveRight.bind(t),"keydown"),this.keycharm.bind("right",this._xStopMoving.bind(t),"keyup"),this.keycharm.bind("=",this._zoomIn.bind(t),"keydown"),this.keycharm.bind("=",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("num+",this._zoomIn.bind(t),"keydown"),this.keycharm.bind("num+",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("num-",this._zoomOut.bind(t),"keydown"),this.keycharm.bind("num-",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("-",this._zoomOut.bind(t),"keydown"),this.keycharm.bind("-",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("[",this._zoomIn.bind(t),"keydown"),this.keycharm.bind("[",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("]",this._zoomOut.bind(t),"keydown"),this.keycharm.bind("]",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("pageup",this._zoomIn.bind(t),"keydown"),this.keycharm.bind("pageup",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("pagedown",this._zoomOut.bind(t),"keydown"),this.keycharm.bind("pagedown",this._stopZoom.bind(t),"keyup")),1==this.constants.dataManipulation.enabled&&(this.keycharm.bind("esc",this._createManipulatorBar.bind(t)),this.keycharm.bind("delete",this._deleteSelected.bind(t)))},s.prototype.destroy=function(){this.start=function(){},this.redraw=function(){},this.timer=!1,this._cleanupPhysicsConfiguration(),this.keycharm.reset(),this.hammer.destroy(),this.off(),this._recursiveDOMDelete(this.containerElement)},s.prototype._recursiveDOMDelete=function(t){for(;1==t.hasChildNodes();)this._recursiveDOMDelete(t.firstChild),t.removeChild(t.firstChild)},s.prototype._getPointer=function(t){return{x:t.x-a.getAbsoluteLeft(this.frame.canvas),y:t.y-a.getAbsoluteTop(this.frame.canvas)}},s.prototype._onTouch=function(t){(new Date).valueOf()-this.touchTime>100&&(this.drag.pointer=this._getPointer(t.center),this.drag.pinched=!1,this.pinch.scale=this._getScale(),this.touchTime=(new Date).valueOf(),this._handleTouch(this.drag.pointer))},s.prototype._onDragStart=function(t){this._handleDragStart(t)},s.prototype._handleDragStart=function(t){void 0===this.drag.pointer&&this._onTouch(t);var e=this._getNodeAt(this.drag.pointer);if(this.drag.dragging=!0,this.drag.selection=[],this.drag.translation=this._getTranslation(),this.drag.nodeId=null,this.draggingNodes=!1,null!=e&&1==this.constants.dragNodes){this.draggingNodes=!0,this.drag.nodeId=e.id,e.isSelected()||this._selectObject(e,!1),this.emit("dragStart",{nodeIds:this.getSelection().nodes});for(var i in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(i)){var s=this.selectionObj.nodes[i],o={id:s.id,node:s,x:s.x,y:s.y,xFixed:s.xFixed,yFixed:s.yFixed};s.xFixed=!0,s.yFixed=!0,this.drag.selection.push(o)}}t.preventDefault()},s.prototype._onDrag=function(t){this._handleOnDrag(t)},s.prototype._handleOnDrag=function(t){if(!this.drag.pinched){this.releaseNode();var e=this._getPointer(t.center),i=this,s=this.drag,o=s.selection;if(o&&o.length&&1==this.constants.dragNodes){var n=e.x-s.pointer.x,r=e.y-s.pointer.y;o.forEach(function(t){var e=t.node;t.xFixed||(e.x=i._XconvertDOMtoCanvas(i._XconvertCanvasToDOM(t.x)+n)),t.yFixed||(e.y=i._YconvertDOMtoCanvas(i._YconvertCanvasToDOM(t.y)+r))}),this.moving||(this.moving=!0,this.start())}else if(1==this.constants.dragNetwork){if(void 0===this.drag.pointer)return void this._handleDragStart(t);var a=e.x-this.drag.pointer.x,h=e.y-this.drag.pointer.y;this._setTranslation(this.drag.translation.x+a,this.drag.translation.y+h),this._redraw()}t.preventDefault()}},s.prototype._onDragEnd=function(t){this._handleDragEnd(t)},s.prototype._handleDragEnd=function(t){this.drag.dragging=!1;var e=this.drag.selection;e&&e.length?(e.forEach(function(t){t.node.xFixed=t.xFixed,t.node.yFixed=t.yFixed}),this.moving=!0,this.start()):this._redraw(),0==this.draggingNodes?this.emit("dragEnd",{nodeIds:[]}):this.emit("dragEnd",{nodeIds:this.getSelection().nodes}),t.preventDefault()},s.prototype._onTap=function(t){var e=this._getPointer(t.center);this.pointerPosition=e,this._handleTap(e)},s.prototype._onDoubleTap=function(t){var e=this._getPointer(t.center);this._handleDoubleTap(e)},s.prototype._onHold=function(t){var e=this._getPointer(t.center);this.pointerPosition=e,this._handleOnHold(e)},s.prototype._onRelease=function(t){var e=this._getPointer(t.center);this._handleOnRelease(e)},s.prototype._onPinch=function(t){var e=this._getPointer(t.center);this.drag.pinched=!0,"scale"in this.pinch||(this.pinch.scale=1);var i=this.pinch.scale*t.scale;this._zoom(i,e)},s.prototype._zoom=function(t,e){if(1==this.constants.zoomable){var i=this._getScale();1e-5>t&&(t=1e-5),t>10&&(t=10);var s=null;void 0!==this.drag&&1==this.drag.dragging&&(s=this.DOMtoCanvas(this.drag.pointer));var o=this._getTranslation(),n=t/i,r=(1-n)*e.x+o.x*n,a=(1-n)*e.y+o.y*n;if(this.areaCenter={x:this._XconvertDOMtoCanvas(e.x),y:this._YconvertDOMtoCanvas(e.y)},this._setScale(t),this._setTranslation(r,a),this.updateClustersDefault(),null!=s){var h=this.canvasToDOM(s);this.drag.pointer.x=h.x,this.drag.pointer.y=h.y}return this._redraw(),t>i?this.emit("zoom",{direction:"+"}):this.emit("zoom",{direction:"-"}),t}},s.prototype._onMouseWheel=function(t){var e=0;if(t.wheelDelta?e=t.wheelDelta/120:t.detail&&(e=-t.detail/3),e){var i=this._getScale(),s=e/10;0>e&&(s/=1-s),i*=1+s;var o=this._getPointer({x:t.pageX,y:t.pageY});this._zoom(i,o)}t.preventDefault()},s.prototype._onMouseMoveTitle=function(t){var e=this._getPointer({x:t.pageX,y:t.pageY});this.popupObj&&this._checkHidePopup(e);var i=this,s=function(){i._checkShowPopup(e)};if(this.popupTimer&&clearInterval(this.popupTimer),this.drag.dragging||(this.popupTimer=setTimeout(s,this.constants.tooltip.delay)),1==this.constants.hover){for(var o in this.hoverObj.edges)this.hoverObj.edges.hasOwnProperty(o)&&(this.hoverObj.edges[o].hover=!1,delete this.hoverObj.edges[o]);var n=this._getNodeAt(e);null==n&&(n=this._getEdgeAt(e)),null!=n&&this._hoverObject(n);for(var r in this.hoverObj.nodes)this.hoverObj.nodes.hasOwnProperty(r)&&(n instanceof f&&n.id!=r||n instanceof g||null==n)&&(this._blurObject(this.hoverObj.nodes[r]),delete this.hoverObj.nodes[r]);this.redraw()}},s.prototype._checkShowPopup=function(t){var e,i={left:this._XconvertDOMtoCanvas(t.x),top:this._YconvertDOMtoCanvas(t.y),right:this._XconvertDOMtoCanvas(t.x),bottom:this._YconvertDOMtoCanvas(t.y)},s=this.popupObj,o=!1;if(void 0==this.popupObj){var n=this.nodes,r=[];for(e in n)if(n.hasOwnProperty(e)){var a=n[e];a.isOverlappingWith(i)&&void 0!==a.getTitle()&&r.push(e)}r.length>0&&(this.popupObj=this.nodes[r[r.length-1]],o=!0)}if(void 0===this.popupObj&&0==o){var h=this.edges,d=[];for(e in h)if(h.hasOwnProperty(e)){var l=h[e];l.connected&&void 0!==l.getTitle()&&l.isOverlappingWith(i)&&d.push(e)}d.length>0&&(this.popupObj=this.edges[d[d.length-1]])}if(this.popupObj){if(this.popupObj!=s){var c=this;c.popup||(c.popup=new v(c.frame,c.constants.tooltip)),c.popup.setPosition(t.x-3,t.y-3),c.popup.setText(c.popupObj.getTitle()),c.popup.show()}}else this.popup&&this.popup.hide()},s.prototype._checkHidePopup=function(t){this.popupObj&&this._getNodeAt(t)||(this.popupObj=void 0,this.popup&&this.popup.hide())},s.prototype.setSize=function(t,e){var i=!1,s=this.frame.canvas.width,o=this.frame.canvas.height;t!=this.constants.width||e!=this.constants.height||this.frame.style.width!=t||this.frame.style.height!=e?(this.frame.style.width=t,this.frame.style.height=e,this.frame.canvas.style.width="100%",this.frame.canvas.style.height="100%",this.frame.canvas.width=this.frame.canvas.clientWidth*this.pixelRatio,this.frame.canvas.height=this.frame.canvas.clientHeight*this.pixelRatio,this.constants.width=t,this.constants.height=e,i=!0):(this.frame.canvas.width!=this.frame.canvas.clientWidth*this.pixelRatio&&(this.frame.canvas.width=this.frame.canvas.clientWidth*this.pixelRatio,i=!0),this.frame.canvas.height!=this.frame.canvas.clientHeight*this.pixelRatio&&(this.frame.canvas.height=this.frame.canvas.clientHeight*this.pixelRatio,i=!0)),1==i&&this.emit("resize",{width:this.frame.canvas.width*this.pixelRatio,height:this.frame.canvas.height*this.pixelRatio,oldWidth:s*this.pixelRatio,oldHeight:o*this.pixelRatio})},s.prototype._setNodes=function(t){var e=this.nodesData;if(t instanceof d||t instanceof l)this.nodesData=t;else if(Array.isArray(t))this.nodesData=new d,this.nodesData.add(t);else{if(t)throw new TypeError("Array or DataSet expected");this.nodesData=new d}if(e&&a.forEach(this.nodesListeners,function(t,i){e.off(i,t)}),this.nodes={},this.nodesData){var i=this;a.forEach(this.nodesListeners,function(t,e){i.nodesData.on(e,t)});var s=this.nodesData.getIds();this._addNodes(s)}this._updateSelection()},s.prototype._addNodes=function(t){for(var e,i=0,s=t.length;s>i;i++){e=t[i];var o=this.nodesData.get(e),n=new f(o,this.images,this.groups,this.constants);if(this.nodes[e]=n,!(0!=n.xFixed&&0!=n.yFixed||null!==n.x&&null!==n.y)){var r=1*t.length+10,a=2*Math.PI*Math.random();0==n.xFixed&&(n.x=r*Math.cos(a)),0==n.yFixed&&(n.y=r*Math.sin(a))}this.moving=!0}this._updateNodeIndexList(),1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout()),this._updateCalculationNodes(),this._reconnectEdges(),this._updateValueRange(this.nodes),this.updateLabels()},s.prototype._updateNodes=function(t,e){for(var i=this.nodes,s=0,o=t.length;o>s;s++){var n=t[s],r=i[n],a=e[s];r?r.setProperties(a,this.constants):(r=new f(properties,this.images,this.groups,this.constants),i[n]=r)}this.moving=!0,1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout()),this._updateNodeIndexList(),this._updateValueRange(i)},s.prototype._removeNodes=function(t){for(var e=this.nodes,i=0,s=t.length;s>i;i++){var o=t[i];delete e[o]}this._updateNodeIndexList(),1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout()),this._updateCalculationNodes(),this._reconnectEdges(),this._updateSelection(),this._updateValueRange(e)},s.prototype._setEdges=function(t){var e=this.edgesData;if(t instanceof d||t instanceof l)this.edgesData=t;else if(Array.isArray(t))this.edgesData=new d,this.edgesData.add(t);else{if(t)throw new TypeError("Array or DataSet expected");this.edgesData=new d}if(e&&a.forEach(this.edgesListeners,function(t,i){e.off(i,t)}),this.edges={},this.edgesData){var i=this;a.forEach(this.edgesListeners,function(t,e){i.edgesData.on(e,t)});var s=this.edgesData.getIds();this._addEdges(s)}this._reconnectEdges()},s.prototype._addEdges=function(t){for(var e=this.edges,i=this.edgesData,s=0,o=t.length;o>s;s++){var n=t[s],r=e[n];r&&r.disconnect();var a=i.get(n,{showInternalIds:!0});e[n]=new g(a,this,this.constants)}this.moving=!0,this._updateValueRange(e),this._createBezierNodes(),this._updateCalculationNodes(),1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout())},s.prototype._updateEdges=function(t){for(var e=this.edges,i=this.edgesData,s=0,o=t.length;o>s;s++){var n=t[s],r=i.get(n),a=e[n];a?(a.disconnect(),a.setProperties(r,this.constants),a.connect()):(a=new g(r,this,this.constants),this.edges[n]=a)}this._createBezierNodes(),1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout()),this.moving=!0,this._updateValueRange(e)},s.prototype._removeEdges=function(t){for(var e=this.edges,i=0,s=t.length;s>i;i++){var o=t[i],n=e[o];n&&(null!=n.via&&delete this.sectors.support.nodes[n.via.id],n.disconnect(),delete e[o])}this.moving=!0,this._updateValueRange(e),1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout()),this._updateCalculationNodes()},s.prototype._reconnectEdges=function(){var t,e=this.nodes,i=this.edges;for(t in e)e.hasOwnProperty(t)&&(e[t].edges=[],e[t].dynamicEdges=[]);for(t in i)if(i.hasOwnProperty(t)){var s=i[t];s.from=null,s.to=null,s.connect()}},s.prototype._updateValueRange=function(t){var e,i=void 0,s=void 0;for(e in t)if(t.hasOwnProperty(e)){var o=t[e].getValue();void 0!==o&&(i=void 0===i?o:Math.min(o,i),s=void 0===s?o:Math.max(o,s))}if(void 0!==i&&void 0!==s)for(e in t)t.hasOwnProperty(e)&&t[e].setValueRange(i,s)},s.prototype.redraw=function(){this.setSize(this.constants.width,this.constants.height),this._redraw()},s.prototype._redraw=function(t){var e=this.frame.canvas.getContext("2d");e.setTransform(this.pixelRatio,0,0,this.pixelRatio,0,0);var i=this.frame.canvas.width*this.pixelRatio,s=this.frame.canvas.height*this.pixelRatio;e.clearRect(0,0,i,s),e.save(),e.translate(this.translation.x,this.translation.y),e.scale(this.scale,this.scale),this.canvasTopLeft={x:this._XconvertDOMtoCanvas(0),y:this._YconvertDOMtoCanvas(0)},this.canvasBottomRight={x:this._XconvertDOMtoCanvas(this.frame.canvas.clientWidth*this.pixelRatio),y:this._YconvertDOMtoCanvas(this.frame.canvas.clientHeight*this.pixelRatio)},1!=t&&(this._doInAllSectors("_drawAllSectorNodes",e),(0==this.drag.dragging||void 0===this.drag.dragging||0==this.constants.hideEdgesOnDrag)&&this._doInAllSectors("_drawEdges",e)),(0==this.drag.dragging||void 0===this.drag.dragging||0==this.constants.hideNodesOnDrag)&&this._doInAllSectors("_drawNodes",e,!1),1!=t&&1==this.controlNodesActive&&this._doInAllSectors("_drawControlNodes",e),e.restore(),1==t&&e.clearRect(0,0,i,s)},s.prototype._setTranslation=function(t,e){void 0===this.translation&&(this.translation={x:0,y:0}),void 0!==t&&(this.translation.x=t),void 0!==e&&(this.translation.y=e),this.emit("viewChanged")},s.prototype._getTranslation=function(){return{x:this.translation.x,y:this.translation.y}},s.prototype._setScale=function(t){this.scale=t},s.prototype._getScale=function(){return this.scale},s.prototype._XconvertDOMtoCanvas=function(t){return(t-this.translation.x)/this.scale},s.prototype._XconvertCanvasToDOM=function(t){return t*this.scale+this.translation.x},s.prototype._YconvertDOMtoCanvas=function(t){return(t-this.translation.y)/this.scale},s.prototype._YconvertCanvasToDOM=function(t){return t*this.scale+this.translation.y},s.prototype.canvasToDOM=function(t){return{x:this._XconvertCanvasToDOM(t.x),y:this._YconvertCanvasToDOM(t.y)}},s.prototype.DOMtoCanvas=function(t){return{x:this._XconvertDOMtoCanvas(t.x),y:this._YconvertDOMtoCanvas(t.y)}},s.prototype._drawNodes=function(t,e){void 0===e&&(e=!1);var i=this.nodes,s=[];for(var o in i)i.hasOwnProperty(o)&&(i[o].setScaleAndPos(this.scale,this.canvasTopLeft,this.canvasBottomRight),i[o].isSelected()?s.push(o):(i[o].inArea()||e)&&i[o].draw(t));for(var n=0,r=s.length;r>n;n++)(i[s[n]].inArea()||e)&&i[s[n]].draw(t)},s.prototype._drawEdges=function(t){var e=this.edges;for(var i in e)if(e.hasOwnProperty(i)){var s=e[i];s.setScale(this.scale),s.connected&&e[i].draw(t)}},s.prototype._drawControlNodes=function(t){var e=this.edges;for(var i in e)e.hasOwnProperty(i)&&e[i]._drawControlNodes(t)},s.prototype._stabilize=function(){1==this.constants.freezeForStabilization&&this._freezeDefinedNodes();for(var t=0;this.moving&&t0)for(t in i)i.hasOwnProperty(t)&&(i[t].discreteStepLimited(e,this.constants.maxVelocity),s=!0);else for(t in i)i.hasOwnProperty(t)&&(i[t].discreteStep(e),s=!0);if(1==s){var o=this.constants.minVelocity/Math.max(this.scale,.05);return o>.5*this.constants.maxVelocity?!0:this._isMoving(o)}return!1},s.prototype._revertPhysicsState=function(){var t=this.nodes;for(var e in t)t.hasOwnProperty(e)&&t[e].revertPosition()},s.prototype._revertPhysicsTick=function(){this._doInAllActiveSectors("_revertPhysicsState"),1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic&&this._doInSupportSector("_revertPhysicsState")},s.prototype._physicsTick=function(){if(!this.freezeSimulation&&1==this.moving){var t=!1,e=!1;this._doInAllActiveSectors("_initializeForceCalculation");var i=this._doInAllActiveSectors("_discreteStepNodes");1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic&&(e=this._doInSupportSector("_discreteStepNodes"));for(var s=0;s2*e||1==this.runDoubleSpeed)&&1==this.moving&&(this._physicsTick(),0!=this.renderTime&&(this.runDoubleSpeed=!0));var i=Date.now();this._redraw(),this.renderTime=Date.now()-i,this.start()},"undefined"!=typeof window&&(window.requestAnimationFrame=window.requestAnimationFrame||window.mozRequestAnimationFrame||window.webkitRequestAnimationFrame||window.msRequestAnimationFrame),s.prototype.start=function(){if(1==this.moving||0!=this.xIncrement||0!=this.yIncrement||0!=this.zoomIncrement)this.timer||(this.timer=1==this.requiresTimeout?window.setTimeout(this._animationStep.bind(this),this.renderTimestep):window.requestAnimationFrame(this._animationStep.bind(this)));else if(this._redraw(),this.stabilizationIterations>1){var t=this,e={iterations:t.stabilizationIterations};this.stabilizationIterations=0,this.startedStabilization=!1,setTimeout(function(){t.emit("stabilized",e)},0)}else this.stabilizationIterations=0},s.prototype._handleNavigation=function(){if(0!=this.xIncrement||0!=this.yIncrement){var t=this._getTranslation();this._setTranslation(t.x+this.xIncrement,t.y+this.yIncrement)}if(0!=this.zoomIncrement){var e={x:this.frame.canvas.clientWidth/2,y:this.frame.canvas.clientHeight/2};this._zoom(this.scale*(1+this.zoomIncrement),e)}},s.prototype.toggleFreeze=function(){0==this.freezeSimulation?this.freezeSimulation=!0:(this.freezeSimulation=!1,this.start())},s.prototype._configureSmoothCurves=function(t){if(void 0===t&&(t=!0),1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic){this._createBezierNodes();for(var e in this.sectors.support.nodes)this.sectors.support.nodes.hasOwnProperty(e)&&void 0===this.edges[this.sectors.support.nodes[e].parentEdgeId]&&delete this.sectors.support.nodes[e]}else{this.sectors.support.nodes={};for(var i in this.edges)this.edges.hasOwnProperty(i)&&(this.edges[i].via=null)}this._updateCalculationNodes(),t||(this.moving=!0,this.start())},s.prototype._createBezierNodes=function(){if(1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic)for(var t in this.edges)if(this.edges.hasOwnProperty(t)){var e=this.edges[t];if(null==e.via){var i="edgeId:".concat(e.id);this.sectors.support.nodes[i]=new f({id:i,mass:1,shape:"circle",image:"",internalMultiplier:1},{},{},this.constants),e.via=this.sectors.support.nodes[i],e.via.parentEdgeId=e.id,e.positionBezierNode()}}},s.prototype._initializeMixinLoaders=function(){for(var t in y)y.hasOwnProperty(t)&&(s.prototype[t]=y[t])},s.prototype.storePosition=function(){console.log("storePosition is deprecated: use .storePositions() from now on."),this.storePositions()},s.prototype.storePositions=function(){var t=[];for(var e in this.nodes)if(this.nodes.hasOwnProperty(e)){var i=this.nodes[e],s=!this.nodes.xFixed,o=!this.nodes.yFixed;(this.nodesData._data[e].x!=Math.round(i.x)||this.nodesData._data[e].y!=Math.round(i.y))&&t.push({id:e,x:Math.round(i.x),y:Math.round(i.y),allowedToMoveX:s,allowedToMoveY:o})}this.nodesData.update(t)},s.prototype.getPositions=function(t){var e={};if(void 0!==t){if(1==Array.isArray(t)){for(var i=0;i=1&&(this.easingTime=0,this._redraw=null!=this.lockedOnNodeId?this._lockedRedraw:this._classicRedraw,this.emit("animationFinished"))},s.prototype._classicRedraw=function(){},s.prototype.isActive=function(){return!this.activator||this.activator.active},s.prototype.setScale=function(){return this._setScale()},s.prototype.getScale=function(){return this._getScale()},s.prototype.getCenterCoordinates=function(){return this.DOMtoCanvas({x:.5*this.frame.canvas.clientWidth,y:.5*this.frame.canvas.clientHeight})},s.prototype.getBoundingBox=function(t){return void 0!==this.nodes[t]?this.nodes[t].boundingBox:void 0},t.exports=s},function(t,e,i){function s(t,e,i){if(!e)throw"No network provided";var s=["edges","physics"],n=o.selectiveBridgeObject(s,i);this.options=n.edges,this.physics=n.physics,this.options.smoothCurves=i.smoothCurves,this.network=e,this.id=void 0,this.fromId=void 0,this.toId=void 0,this.title=void 0,this.widthSelected=this.options.width*this.options.widthSelectionMultiplier,this.value=void 0,this.selected=!1,this.hover=!1,this.labelDimensions={top:0,left:0,width:0,height:0,yLine:0},this.dirtyLabel=!0,this.from=null,this.to=null,this.via=null,this.fromBackup=null,this.toBackup=null,this.originalFromId=[],this.originalToId=[],this.connected=!1,this.widthFixed=!1,this.lengthFixed=!1,this.setProperties(t),this.controlNodesEnabled=!1,this.controlNodes={from:null,to:null,positions:{}},this.connectedNode=null}var o=i(1),n=i(40);s.prototype.setProperties=function(t){if(t){var e=["style","fontSize","fontFace","fontColor","fontFill","fontStrokeWidth","fontStrokeColor","width","widthSelectionMultiplier","hoverWidth","arrowScaleFactor","dash","inheritColor","labelAlignment"];switch(o.selectiveDeepExtend(e,this.options,t),void 0!==t.from&&(this.fromId=t.from),void 0!==t.to&&(this.toId=t.to),void 0!==t.id&&(this.id=t.id),void 0!==t.label&&(this.label=t.label,this.dirtyLabel=!0),void 0!==t.title&&(this.title=t.title),void 0!==t.value&&(this.value=t.value),void 0!==t.length&&(this.physics.springLength=t.length),void 0!==t.color&&(this.options.inheritColor=!1,o.isString(t.color)?(this.options.color.color=t.color,this.options.color.highlight=t.color):(void 0!==t.color.color&&(this.options.color.color=t.color.color),void 0!==t.color.highlight&&(this.options.color.highlight=t.color.highlight),void 0!==t.color.hover&&(this.options.color.hover=t.color.hover))),this.connect(),this.widthFixed=this.widthFixed||void 0!==t.width,this.lengthFixed=this.lengthFixed||void 0!==t.length,this.widthSelected=this.options.width*this.options.widthSelectionMultiplier,this.options.style){case"line":this.draw=this._drawLine;break;case"arrow":this.draw=this._drawArrow;break;case"arrow-center":this.draw=this._drawArrowCenter;break;case"dash-line":this.draw=this._drawDashLine;break;default:this.draw=this._drawLine}}},s.prototype.connect=function(){this.disconnect(),this.from=this.network.nodes[this.fromId]||null,this.to=this.network.nodes[this.toId]||null,this.connected=this.from&&this.to,this.connected?(this.from.attachEdge(this),this.to.attachEdge(this)):(this.from&&this.from.detachEdge(this),this.to&&this.to.detachEdge(this))},s.prototype.disconnect=function(){this.from&&(this.from.detachEdge(this),this.from=null),this.to&&(this.to.detachEdge(this),this.to=null),this.connected=!1},s.prototype.getTitle=function(){return"function"==typeof this.title?this.title():this.title -},s.prototype.getValue=function(){return this.value},s.prototype.setValueRange=function(t,e){if(!this.widthFixed&&void 0!==this.value){var i=(this.options.widthMax-this.options.widthMin)/(e-t);this.options.width=(this.value-t)*i+this.options.widthMin,this.widthSelected=this.options.width*this.options.widthSelectionMultiplier}},s.prototype.draw=function(){throw"Method draw not initialized in edge"},s.prototype.isOverlappingWith=function(t){if(this.connected){var e=10,i=this.from.x,s=this.from.y,o=this.to.x,n=this.to.y,r=t.left,a=t.top,h=this._getDistanceToEdge(i,s,o,n,r,a);return e>h}return!1},s.prototype._getColor=function(){var t=this.options.color;return"to"==this.options.inheritColor?t={highlight:this.to.options.color.highlight.border,hover:this.to.options.color.hover.border,color:this.to.options.color.border}:("from"==this.options.inheritColor||1==this.options.inheritColor)&&(t={highlight:this.from.options.color.highlight.border,hover:this.from.options.color.hover.border,color:this.from.options.color.border}),1==this.selected?t.highlight:1==this.hover?t.hover:t.color},s.prototype._drawLine=function(t){if(t.strokeStyle=this._getColor(),t.lineWidth=this._getLineWidth(),this.from!=this.to){var e,i=this._line(t);if(this.label){if(1==this.options.smoothCurves.enabled&&null!=i){var s=.5*(.5*(this.from.x+i.x)+.5*(this.to.x+i.x)),o=.5*(.5*(this.from.y+i.y)+.5*(this.to.y+i.y));e={x:s,y:o}}else e=this._pointOnLine(.5);this._label(t,this.label,e.x,e.y)}}else{var n,r,a=this.physics.springLength/4,h=this.from;h.width||h.resize(t),h.width>h.height?(n=h.x+h.width/2,r=h.y-a):(n=h.x+a,r=h.y-h.height/2),this._circle(t,n,r,a),e=this._pointOnCircle(n,r,a,.5),this._label(t,this.label,e.x,e.y)}},s.prototype._getLineWidth=function(){return 1==this.selected?Math.max(Math.min(this.widthSelected,this.options.widthMax),.3*this.networkScaleInv):1==this.hover?Math.max(Math.min(this.options.hoverWidth,this.options.widthMax),.3*this.networkScaleInv):Math.max(this.options.width,.3*this.networkScaleInv)},s.prototype._getViaCoordinates=function(){if(1==this.options.smoothCurves.dynamic&&1==this.options.smoothCurves.enabled)return this.via;if(0==this.options.smoothCurves.enabled)return{x:0,y:0};var t=null,e=null,i=this.options.smoothCurves.roundness,s=this.options.smoothCurves.type,o=Math.abs(this.from.x-this.to.x),n=Math.abs(this.from.y-this.to.y);return"discrete"==s||"diagonalCross"==s?Math.abs(this.from.x-this.to.x)this.to.y?this.from.xthis.to.x&&(t=this.from.x-i*n,e=this.from.y-i*n):this.from.ythis.to.x&&(t=this.from.x-i*n,e=this.from.y+i*n)),"discrete"==s&&(t=i*n>o?this.from.x:t)):Math.abs(this.from.x-this.to.x)>Math.abs(this.from.y-this.to.y)&&(this.from.y>this.to.y?this.from.xthis.to.x&&(t=this.from.x-i*o,e=this.from.y-i*o):this.from.ythis.to.x&&(t=this.from.x-i*o,e=this.from.y+i*o)),"discrete"==s&&(e=i*o>n?this.from.y:e)):"straightCross"==s?Math.abs(this.from.x-this.to.x)Math.abs(this.from.y-this.to.y)&&(t=this.from.xthis.to.y?this.from.xthis.to.x&&(t=this.from.x-i*n,e=this.from.y-i*n,t=this.to.x>t?this.to.x:t):this.from.ythis.to.x&&(t=this.from.x-i*n,e=this.from.y+i*n,t=this.to.x>t?this.to.x:t)):Math.abs(this.from.x-this.to.x)>Math.abs(this.from.y-this.to.y)&&(this.from.y>this.to.y?this.from.xe?this.to.y:e):this.from.x>this.to.x&&(t=this.from.x-i*o,e=this.from.y-i*o,e=this.to.y>e?this.to.y:e):this.from.ythis.to.x&&(t=this.from.x-i*o,e=this.from.y+i*o,e=this.to.yd;d++){var l=t.measureText(n[d]).width;h=l>h?l:h}var c=this.options.fontSize*r,p=i-h/2,u=s-c/2;this.labelDimensions={top:u,left:p,width:h,height:c,yLine:o}}var o=this.labelDimensions.yLine;t.save(),"horizontal"!=this.options.labelAlignment&&(t.translate(i,o),this._rotateForLabelAlignment(t),i=0,o=0),this._drawLabelRect(t),this._drawLabelText(t,i,o,n,r,a),t.restore()}},s.prototype._rotateForLabelAlignment=function(t){var e=this.from.y-this.to.y,i=this.from.x-this.to.x,s=Math.atan2(e,i);(-1>s&&0>i||s>0&&0>i)&&(s+=Math.PI),t.rotate(s)},s.prototype._drawLabelRect=function(t){if(void 0!==this.options.fontFill&&null!==this.options.fontFill&&"none"!==this.options.fontFill){t.fillStyle=this.options.fontFill;var e=2;"line-center"==this.options.labelAlignment?t.fillRect(.5*-this.labelDimensions.width,.5*-this.labelDimensions.height,this.labelDimensions.width,this.labelDimensions.height):"line-above"==this.options.labelAlignment?t.fillRect(.5*-this.labelDimensions.width,-(this.labelDimensions.height+e),this.labelDimensions.width,this.labelDimensions.height):"line-below"==this.options.labelAlignment?t.fillRect(.5*-this.labelDimensions.width,e,this.labelDimensions.width,this.labelDimensions.height):t.fillRect(this.labelDimensions.left,this.labelDimensions.top,this.labelDimensions.width,this.labelDimensions.height)}},s.prototype._drawLabelText=function(t,e,i,s,o,n){if(t.fillStyle=this.options.fontColor||"black",t.textAlign="center","horizontal"!=this.options.labelAlignment){var r=2;"line-above"==this.options.labelAlignment?(t.textBaseline="alphabetic",i-=2*r):"line-below"==this.options.labelAlignment?(t.textBaseline="hanging",i+=2*r):t.textBaseline="middle"}else t.textBaseline="middle";this.options.fontStrokeWidth>0&&(t.lineWidth=this.options.fontStrokeWidth,t.strokeStyle=this.options.fontStrokeColor,t.lineJoin="round");for(var a=0;o>a;a++)this.options.fontStrokeWidth>0&&t.strokeText(s[a],e,i),t.fillText(s[a],e,i),i+=n},s.prototype._drawDashLine=function(t){t.strokeStyle=this._getColor(),t.lineWidth=this._getLineWidth();var e=null;if(void 0!==t.setLineDash){t.save();var i=[0];i=void 0!==this.options.dash.length&&void 0!==this.options.dash.gap?[this.options.dash.length,this.options.dash.gap]:[5,5],t.setLineDash(i),t.lineDashOffset=0,e=this._line(t),t.setLineDash([0]),t.lineDashOffset=0,t.restore()}else t.beginPath(),t.lineCap="round",void 0!==this.options.dash.altLength?t.dashedLine(this.from.x,this.from.y,this.to.x,this.to.y,[this.options.dash.length,this.options.dash.gap,this.options.dash.altLength,this.options.dash.gap]):void 0!==this.options.dash.length&&void 0!==this.options.dash.gap?t.dashedLine(this.from.x,this.from.y,this.to.x,this.to.y,[this.options.dash.length,this.options.dash.gap]):(t.moveTo(this.from.x,this.from.y),t.lineTo(this.to.x,this.to.y)),t.stroke();if(this.label){var s;if(1==this.options.smoothCurves.enabled&&null!=e){var o=.5*(.5*(this.from.x+e.x)+.5*(this.to.x+e.x)),n=.5*(.5*(this.from.y+e.y)+.5*(this.to.y+e.y));s={x:o,y:n}}else s=this._pointOnLine(.5);this._label(t,this.label,s.x,s.y)}},s.prototype._pointOnLine=function(t){return{x:(1-t)*this.from.x+t*this.to.x,y:(1-t)*this.from.y+t*this.to.y}},s.prototype._pointOnCircle=function(t,e,i,s){var o=2*(s-3/8)*Math.PI;return{x:t+i*Math.cos(o),y:e-i*Math.sin(o)}},s.prototype._drawArrowCenter=function(t){var e;if(t.strokeStyle=this._getColor(),t.fillStyle=t.strokeStyle,t.lineWidth=this._getLineWidth(),this.from!=this.to){var i=this._line(t),s=Math.atan2(this.to.y-this.from.y,this.to.x-this.from.x),o=(10+5*this.options.width)*this.options.arrowScaleFactor;if(1==this.options.smoothCurves.enabled&&null!=i){var n=.5*(.5*(this.from.x+i.x)+.5*(this.to.x+i.x)),r=.5*(.5*(this.from.y+i.y)+.5*(this.to.y+i.y));e={x:n,y:r}}else e=this._pointOnLine(.5);t.arrow(e.x,e.y,s,o),t.fill(),t.stroke(),this.label&&this._label(t,this.label,e.x,e.y)}else{var a,h,d=.25*Math.max(100,this.physics.springLength),l=this.from;l.width||l.resize(t),l.width>l.height?(a=l.x+.5*l.width,h=l.y-d):(a=l.x+d,h=l.y-.5*l.height),this._circle(t,a,h,d);var s=.2*Math.PI,o=(10+5*this.options.width)*this.options.arrowScaleFactor;e=this._pointOnCircle(a,h,d,.5),t.arrow(e.x,e.y,s,o),t.fill(),t.stroke(),this.label&&(e=this._pointOnCircle(a,h,d,.5),this._label(t,this.label,e.x,e.y))}},s.prototype._pointOnBezier=function(t){var e=this._getViaCoordinates(),i=Math.pow(1-t,2)*this.from.x+2*t*(1-t)*e.x+Math.pow(t,2)*this.to.x,s=Math.pow(1-t,2)*this.from.y+2*t*(1-t)*e.y+Math.pow(t,2)*this.to.y;return{x:i,y:s}},s.prototype._findBorderPosition=function(t,e){var i,s,o,n,r,a=10,h=0,d=0,l=1,c=.2,p=this.to;for(1==t&&(p=this.from);l>=d&&a>h;){var u=.5*(d+l);if(i=this._pointOnBezier(u),s=Math.atan2(p.y-i.y,p.x-i.x),o=p.distanceToBorder(e,s),n=Math.sqrt(Math.pow(i.x-p.x,2)+Math.pow(i.y-p.y,2)),r=o-n,Math.abs(r)r?0==t?d=u:l=u:0==t?l=u:d=u,h++}return i.t=u,i},s.prototype._drawArrow=function(t){t.strokeStyle=this._getColor(),t.fillStyle=t.strokeStyle,t.lineWidth=this._getLineWidth();var e,i,s;if(this.from!=this.to){if(this._line(t),1==this.options.smoothCurves.enabled){var o=this._getViaCoordinates();s=this._findBorderPosition(!1,t);var n=this._pointOnBezier(Math.max(0,s.t-.1));e=Math.atan2(s.y-n.y,s.x-n.x)}else{e=Math.atan2(this.to.y-this.from.y,this.to.x-this.from.x);var r=this.to.x-this.from.x,a=this.to.y-this.from.y,h=Math.sqrt(r*r+a*a),d=this.to.distanceToBorder(t,e),l=(h-d)/h;s={},s.x=(1-l)*this.from.x+l*this.to.x,s.y=(1-l)*this.from.y+l*this.to.y}if(i=(10+5*this.options.width)*this.options.arrowScaleFactor,t.arrow(s.x,s.y,e,i),t.fill(),t.stroke(),this.label){var c;c=1==this.options.smoothCurves.enabled&&null!=o?this._pointOnBezier(.5):this._pointOnLine(.5),this._label(t,this.label,c.x,c.y)}}else{var p,u,m,f=this.from,g=.25*Math.max(100,this.physics.springLength);f.width||f.resize(t),f.width>f.height?(p=f.x+.5*f.width,u=f.y-g,m={x:p,y:f.y,angle:.9*Math.PI}):(p=f.x+g,u=f.y-.5*f.height,m={x:f.x,y:u,angle:.6*Math.PI}),t.beginPath(),t.arc(p,u,g,0,2*Math.PI,!1),t.stroke();var i=(10+5*this.options.width)*this.options.arrowScaleFactor;t.arrow(m.x,m.y,m.angle,i),t.fill(),t.stroke(),this.label&&(c=this._pointOnCircle(p,u,g,.5),this._label(t,this.label,c.x,c.y))}},s.prototype._getDistanceToEdge=function(t,e,i,s,o,n){var r=0;if(this.from!=this.to)if(1==this.options.smoothCurves.enabled){var a,h;if(1==this.options.smoothCurves.enabled&&1==this.options.smoothCurves.dynamic)a=this.via.x,h=this.via.y;else{var d=this._getViaCoordinates();a=d.x,h=d.y}var l,c,p,u,m,f,g,v=1e9;for(c=0;10>c;c++)p=.1*c,u=Math.pow(1-p,2)*t+2*p*(1-p)*a+Math.pow(p,2)*i,m=Math.pow(1-p,2)*e+2*p*(1-p)*h+Math.pow(p,2)*s,c>0&&(l=this._getDistanceToLine(f,g,u,m,o,n),v=v>l?l:v),f=u,g=m;r=v}else r=this._getDistanceToLine(t,e,i,s,o,n);else{var u,m,y,b,_=.25*this.physics.springLength,x=this.from;x.width>x.height?(u=x.x+.5*x.width,m=x.y-_):(u=x.x+_,m=x.y-.5*x.height),y=u-o,b=m-n,r=Math.abs(Math.sqrt(y*y+b*b)-_)}return this.labelDimensions.lefto&&this.labelDimensions.topn?0:r},s.prototype._getDistanceToLine=function(t,e,i,s,o,n){var r=i-t,a=s-e,h=r*r+a*a,d=((o-t)*r+(n-e)*a)/h;d>1?d=1:0>d&&(d=0);var l=t+d*r,c=e+d*a,p=l-o,u=c-n;return Math.sqrt(p*p+u*u)},s.prototype.setScale=function(t){this.networkScaleInv=1/t},s.prototype.select=function(){this.selected=!0},s.prototype.unselect=function(){this.selected=!1},s.prototype.positionBezierNode=function(){null!==this.via&&null!==this.from&&null!==this.to?(this.via.x=.5*(this.from.x+this.to.x),this.via.y=.5*(this.from.y+this.to.y)):(this.via.x=0,this.via.y=0)},s.prototype._drawControlNodes=function(t){if(1==this.controlNodesEnabled){if(null===this.controlNodes.from&&null===this.controlNodes.to){var e="edgeIdFrom:".concat(this.id),i="edgeIdTo:".concat(this.id),s={nodes:{group:"",radius:7,borderWidth:2,borderWidthSelected:2},physics:{damping:0},clustering:{maxNodeSizeIncrements:0,nodeScaling:{width:0,height:0,radius:0}}};this.controlNodes.from=new n({id:e,shape:"dot",color:{background:"#ff0000",border:"#3c3c3c",highlight:{background:"#07f968"}}},{},{},s),this.controlNodes.to=new n({id:i,shape:"dot",color:{background:"#ff0000",border:"#3c3c3c",highlight:{background:"#07f968"}}},{},{},s)}this.controlNodes.positions={},0==this.controlNodes.from.selected&&(this.controlNodes.positions.from=this.getControlNodeFromPosition(t),this.controlNodes.from.x=this.controlNodes.positions.from.x,this.controlNodes.from.y=this.controlNodes.positions.from.y),0==this.controlNodes.to.selected&&(this.controlNodes.positions.to=this.getControlNodeToPosition(t),this.controlNodes.to.x=this.controlNodes.positions.to.x,this.controlNodes.to.y=this.controlNodes.positions.to.y),this.controlNodes.from.draw(t),this.controlNodes.to.draw(t)}else this.controlNodes={from:null,to:null,positions:{}}},s.prototype._enableControlNodes=function(){this.fromBackup=this.from,this.toBackup=this.to,this.controlNodesEnabled=!0},s.prototype._disableControlNodes=function(){this.fromId=this.from.id,this.toId=this.to.id,this.fromId!=this.fromBackup.id?this.fromBackup.detachEdge(this):this.toId!=this.toBackup.id&&this.toBackup.detachEdge(this),this.fromBackup=null,this.toBackup=null,this.controlNodesEnabled=!1},s.prototype._getSelectedControlNode=function(t,e){var i=this.controlNodes.positions,s=Math.sqrt(Math.pow(t-i.from.x,2)+Math.pow(e-i.from.y,2)),o=Math.sqrt(Math.pow(t-i.to.x,2)+Math.pow(e-i.to.y,2));return 15>s?(this.connectedNode=this.from,this.from=this.controlNodes.from,this.controlNodes.from):15>o?(this.connectedNode=this.to,this.to=this.controlNodes.to,this.controlNodes.to):null},s.prototype._restoreControlNodes=function(){1==this.controlNodes.from.selected?(this.from=this.connectedNode,this.connectedNode=null,this.controlNodes.from.unselect()):1==this.controlNodes.to.selected&&(this.to=this.connectedNode,this.connectedNode=null,this.controlNodes.to.unselect())},s.prototype.getControlNodeFromPosition=function(t){var e;if(1==this.options.smoothCurves.enabled)e=this._findBorderPosition(!0,t);else{var i=Math.atan2(this.to.y-this.from.y,this.to.x-this.from.x),s=this.to.x-this.from.x,o=this.to.y-this.from.y,n=Math.sqrt(s*s+o*o),r=this.from.distanceToBorder(t,i+Math.PI),a=(n-r)/n;e={},e.x=a*this.from.x+(1-a)*this.to.x,e.y=a*this.from.y+(1-a)*this.to.y}return e},s.prototype.getControlNodeToPosition=function(t){var e;if(1==this.options.smoothCurves.enabled)e=this._findBorderPosition(!1,t);else{var i=Math.atan2(this.to.y-this.from.y,this.to.x-this.from.x),s=this.to.x-this.from.x,o=this.to.y-this.from.y,n=Math.sqrt(s*s+o*o),r=this.to.distanceToBorder(t,i),a=(n-r)/n;e={},e.x=(1-a)*this.from.x+a*this.to.x,e.y=(1-a)*this.from.y+a*this.to.y}return e},t.exports=s},function(t,e,i){function s(){this.clear(),this.defaultIndex=0}i(1);s.DEFAULT=[{border:"#2B7CE9",background:"#97C2FC",highlight:{border:"#2B7CE9",background:"#D2E5FF"},hover:{border:"#2B7CE9",background:"#D2E5FF"}},{border:"#FFA500",background:"#FFFF00",highlight:{border:"#FFA500",background:"#FFFFA3"},hover:{border:"#FFA500",background:"#FFFFA3"}},{border:"#FA0A10",background:"#FB7E81",highlight:{border:"#FA0A10",background:"#FFAFB1"},hover:{border:"#FA0A10",background:"#FFAFB1"}},{border:"#41A906",background:"#7BE141",highlight:{border:"#41A906",background:"#A1EC76"},hover:{border:"#41A906",background:"#A1EC76"}},{border:"#E129F0",background:"#EB7DF4",highlight:{border:"#E129F0",background:"#F0B3F5"},hover:{border:"#E129F0",background:"#F0B3F5"}},{border:"#7C29F0",background:"#AD85E4",highlight:{border:"#7C29F0",background:"#D3BDF0"},hover:{border:"#7C29F0",background:"#D3BDF0"}},{border:"#C37F00",background:"#FFA807",highlight:{border:"#C37F00",background:"#FFCA66"},hover:{border:"#C37F00",background:"#FFCA66"}},{border:"#4220FB",background:"#6E6EFD",highlight:{border:"#4220FB",background:"#9B9BFD"},hover:{border:"#4220FB",background:"#9B9BFD"}},{border:"#FD5A77",background:"#FFC0CB",highlight:{border:"#FD5A77",background:"#FFD1D9"},hover:{border:"#FD5A77",background:"#FFD1D9"}},{border:"#4AD63A",background:"#C2FABC",highlight:{border:"#4AD63A",background:"#E6FFE3"},hover:{border:"#4AD63A",background:"#E6FFE3"}}],s.prototype.clear=function(){this.groups={},this.groups.length=function(){var t=0;for(var e in this)this.hasOwnProperty(e)&&t++;return t}},s.prototype.get=function(t){var e=this.groups[t];if(void 0==e){var i=this.defaultIndex%s.DEFAULT.length;this.defaultIndex++,e={},e.color=s.DEFAULT[i],this.groups[t]=e}return e},s.prototype.add=function(t,e){return this.groups[t]=e,e},t.exports=s},function(t){function e(){this.images={},this.imageBroken={},this.callback=void 0}e.prototype.setOnloadCallback=function(t){this.callback=t},e.prototype.load=function(t,e){var i=this.images[t];if(void 0===i){var s=this;i=new Image,i.onload=function(){0==this.width&&(document.body.appendChild(this),this.width=this.offsetWidth,this.height=this.offsetHeight,document.body.removeChild(this)),s.callback&&(s.images[t]=i,s.callback(this))},i.onerror=function(){void 0===e?(console.error("Could not load image:",t),delete this.src,s.callback&&s.callback(this)):s.imageBroken[t]===!0?(console.error("Could not load brokenImage:",e),delete this.src,s.callback&&s.callback(this)):(this.src=e,s.imageBroken[t]=!0)},i.src=t}return i},t.exports=e},function(t,e,i){function s(t,e,i,s){var n=o.selectiveBridgeObject(["nodes"],s);this.options=n.nodes,this.selected=!1,this.hover=!1,this.edges=[],this.dynamicEdges=[],this.reroutedEdges={},this.fontDrawThreshold=3,this.id=void 0,this.allowedToMoveX=!1,this.allowedToMoveY=!1,this.xFixed=!1,this.yFixed=!1,this.horizontalAlignLeft=!0,this.verticalAlignTop=!0,this.baseRadiusValue=s.nodes.radius,this.radiusFixed=!1,this.level=-1,this.preassignedLevel=!1,this.hierarchyEnumerated=!1,this.labelDimensions={top:0,left:0,width:0,height:0,yLine:0},this.boundingBox={top:0,left:0,right:0,bottom:0},this.imagelist=e,this.grouplist=i,this.fx=0,this.fy=0,this.vx=0,this.vy=0,this.x=null,this.y=null,this.previousState={vx:0,vy:0,x:0,y:0},this.damping=s.physics.damping,this.fixedData={x:null,y:null},this.setProperties(t,n),this.resetCluster(),this.dynamicEdgesLength=0,this.clusterSession=0,this.clusterSizeWidthFactor=s.clustering.nodeScaling.width,this.clusterSizeHeightFactor=s.clustering.nodeScaling.height,this.clusterSizeRadiusFactor=s.clustering.nodeScaling.radius,this.maxNodeSizeIncrements=s.clustering.maxNodeSizeIncrements,this.growthIndicator=0,this.networkScaleInv=1,this.networkScale=1,this.canvasTopLeft={x:-300,y:-300},this.canvasBottomRight={x:300,y:300},this.parentEdgeId=null}var o=i(1);s.prototype.revertPosition=function(){this.x=this.previousState.x,this.y=this.previousState.y,this.vx=this.previousState.vx,this.vy=this.previousState.vy},s.prototype.resetCluster=function(){this.formationScale=void 0,this.clusterSize=1,this.containedNodes={},this.containedEdges={},this.clusterSessions=[]},s.prototype.attachEdge=function(t){-1==this.edges.indexOf(t)&&this.edges.push(t),-1==this.dynamicEdges.indexOf(t)&&this.dynamicEdges.push(t),this.dynamicEdgesLength=this.dynamicEdges.length},s.prototype.detachEdge=function(t){var e=this.edges.indexOf(t);-1!=e&&this.edges.splice(e,1),e=this.dynamicEdges.indexOf(t),-1!=e&&this.dynamicEdges.splice(e,1),this.dynamicEdgesLength=this.dynamicEdges.length},s.prototype.setProperties=function(t,e){if(t){var i=["borderWidth","borderWidthSelected","shape","image","brokenImage","radius","fontColor","fontSize","fontFace","fontFill","fontStrokeWidth","fontStrokeColor","group","mass"];if(o.selectiveDeepExtend(i,this.options,t),void 0!==t.id&&(this.id=t.id),void 0!==t.label&&(this.label=t.label,this.originalLabel=t.label),void 0!==t.title&&(this.title=t.title),void 0!==t.x&&(this.x=t.x),void 0!==t.y&&(this.y=t.y),void 0!==t.value&&(this.value=t.value),void 0!==t.level&&(this.level=t.level,this.preassignedLevel=!0),void 0!==t.horizontalAlignLeft&&(this.horizontalAlignLeft=t.horizontalAlignLeft),void 0!==t.verticalAlignTop&&(this.verticalAlignTop=t.verticalAlignTop),void 0!==t.triggerFunction&&(this.triggerFunction=t.triggerFunction),void 0===this.id)throw"Node must have an id";if("number"==typeof this.options.group||"string"==typeof this.options.group&&""!=this.options.group){var s=this.grouplist.get(this.options.group);o.deepExtend(this.options,s),this.options.color=o.parseColor(this.options.color)}if(void 0!==t.radius&&(this.baseRadiusValue=this.options.radius),void 0!==t.color&&(this.options.color=o.parseColor(t.color)),void 0!==this.options.image&&""!=this.options.image){if(!this.imagelist)throw"No imagelist provided";this.imageObj=this.imagelist.load(this.options.image,this.options.brokenImage)}switch(void 0!==t.allowedToMoveX?(this.xFixed=!t.allowedToMoveX,this.allowedToMoveX=t.allowedToMoveX):void 0!==t.x&&0==this.allowedToMoveX&&(this.xFixed=!0),void 0!==t.allowedToMoveY?(this.yFixed=!t.allowedToMoveY,this.allowedToMoveY=t.allowedToMoveY):void 0!==t.y&&0==this.allowedToMoveY&&(this.yFixed=!0),this.radiusFixed=this.radiusFixed||void 0!==t.radius,("image"===this.options.shape||"circularImage"===this.options.shape)&&(this.options.radiusMin=e.nodes.widthMin,this.options.radiusMax=e.nodes.widthMax),this.options.shape){case"database":this.draw=this._drawDatabase,this.resize=this._resizeDatabase;break;case"box":this.draw=this._drawBox,this.resize=this._resizeBox;break;case"circle":this.draw=this._drawCircle,this.resize=this._resizeCircle;break;case"ellipse":this.draw=this._drawEllipse,this.resize=this._resizeEllipse;break;case"image":this.draw=this._drawImage,this.resize=this._resizeImage;break;case"circularImage":this.draw=this._drawCircularImage,this.resize=this._resizeCircularImage;break;case"text":this.draw=this._drawText,this.resize=this._resizeText;break;case"dot":this.draw=this._drawDot,this.resize=this._resizeShape;break;case"square":this.draw=this._drawSquare,this.resize=this._resizeShape;break;case"triangle":this.draw=this._drawTriangle,this.resize=this._resizeShape;break;case"triangleDown":this.draw=this._drawTriangleDown,this.resize=this._resizeShape;break;case"star":this.draw=this._drawStar,this.resize=this._resizeShape;break;default:this.draw=this._drawEllipse,this.resize=this._resizeEllipse}this._reset()}},s.prototype.select=function(){this.selected=!0,this._reset()},s.prototype.unselect=function(){this.selected=!1,this._reset()},s.prototype.clearSizeCache=function(){this._reset()},s.prototype._reset=function(){this.width=void 0,this.height=void 0},s.prototype.getTitle=function(){return"function"==typeof this.title?this.title():this.title},s.prototype.distanceToBorder=function(t,e){var i=1;switch(this.width||this.resize(t),this.options.shape){case"circle":case"dot":return this.options.radius+i;case"ellipse":var s=this.width/2,o=this.height/2,n=Math.sin(e)*s,r=Math.cos(e)*o;return s*o/Math.sqrt(n*n+r*r);case"box":case"image":case"text":default:return this.width?Math.min(Math.abs(this.width/2/Math.cos(e)),Math.abs(this.height/2/Math.sin(e)))+i:0}},s.prototype._setForce=function(t,e){this.fx=t,this.fy=e},s.prototype._addForce=function(t,e){this.fx+=t,this.fy+=e},s.prototype.storeState=function(){this.previousState.x=this.x,this.previousState.y=this.y,this.previousState.vx=this.vx,this.previousState.vy=this.vy},s.prototype.discreteStep=function(t){if(this.storeState(),this.xFixed)this.fx=0,this.vx=0;else{var e=this.damping*this.vx,i=(this.fx-e)/this.options.mass;this.vx+=i*t,this.x+=this.vx*t}if(this.yFixed)this.fy=0,this.vy=0;else{var s=this.damping*this.vy,o=(this.fy-s)/this.options.mass;this.vy+=o*t,this.y+=this.vy*t}},s.prototype.discreteStepLimited=function(t,e){if(this.storeState(),this.xFixed)this.fx=0,this.vx=0;else{var i=this.damping*this.vx,s=(this.fx-i)/this.options.mass;this.vx+=s*t,this.vx=Math.abs(this.vx)>e?this.vx>0?e:-e:this.vx,this.x+=this.vx*t}if(this.yFixed)this.fy=0,this.vy=0;else{var o=this.damping*this.vy,n=(this.fy-o)/this.options.mass;this.vy+=n*t,this.vy=Math.abs(this.vy)>e?this.vy>0?e:-e:this.vy,this.y+=this.vy*t}},s.prototype.isFixed=function(){return this.xFixed&&this.yFixed},s.prototype.isMoving=function(t){var e=Math.sqrt(Math.pow(this.vx,2)+Math.pow(this.vy,2));return e>t},s.prototype.isSelected=function(){return this.selected},s.prototype.getValue=function(){return this.value},s.prototype.getDistance=function(t,e){var i=this.x-t,s=this.y-e;return Math.sqrt(i*i+s*s)},s.prototype.setValueRange=function(t,e){if(!this.radiusFixed&&void 0!==this.value)if(e==t)this.options.radius=(this.options.radiusMin+this.options.radiusMax)/2;else{var i=(this.options.radiusMax-this.options.radiusMin)/(e-t);this.options.radius=(this.value-t)*i+this.options.radiusMin}this.baseRadiusValue=this.options.radius},s.prototype.draw=function(){throw"Draw method not initialized for node"},s.prototype.resize=function(){throw"Resize method not initialized for node"},s.prototype.isOverlappingWith=function(t){return this.leftt.left&&this.topt.top},s.prototype._resizeImage=function(){if(!this.width||!this.height){var t,e;if(this.value){this.options.radius=this.baseRadiusValue;var i=this.imageObj.height/this.imageObj.width;void 0!==i?(t=this.options.radius||this.imageObj.width,e=this.options.radius*i||this.imageObj.height):(t=0,e=0)}else t=this.imageObj.width,e=this.imageObj.height;this.width=t,this.height=e,this.growthIndicator=0,this.width>0&&this.height>0&&(this.width+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeWidthFactor,this.height+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeHeightFactor,this.options.radius+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.width-t)}},s.prototype._drawImageAtPosition=function(t){if(0!=this.imageObj.width){if(this.clusterSize>1){var e=this.clusterSize>1?10:0;e*=this.networkScaleInv,e=Math.min(.2*this.width,e),t.globalAlpha=.5,t.drawImage(this.imageObj,this.left-e,this.top-e,this.width+2*e,this.height+2*e)}t.globalAlpha=1,t.drawImage(this.imageObj,this.left,this.top,this.width,this.height)}},s.prototype._drawImageLabel=function(t){var e,i=0;if(this.height){i=this.height/2;var s=this.getTextSize(t);s.lineCount>=1&&(i+=s.height/2,i+=3)}e=this.y+i,this._label(t,this.label,this.x,e,void 0)},s.prototype._drawImage=function(t){this._resizeImage(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2,this._drawImageAtPosition(t),this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height,this._drawImageLabel(t),this.boundingBox.left=Math.min(this.boundingBox.left,this.labelDimensions.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelDimensions.left+this.labelDimensions.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelDimensions.height)},s.prototype._resizeCircularImage=function(t){if(this.imageObj.src&&this.imageObj.width&&this.imageObj.height)this._swapToImageResizeWhenImageLoaded&&(this.width=0,this.height=0,delete this._swapToImageResizeWhenImageLoaded),this._resizeImage(t);else if(!this.width){var e=2*this.options.radius;this.width=e,this.height=e,this.options.radius+=.5*Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.options.radius-.5*e,this._swapToImageResizeWhenImageLoaded=!0}},s.prototype._drawCircularImage=function(t){this._resizeCircularImage(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2;var e=this.left+this.width/2,i=this.top+this.height/2,s=Math.abs(this.height/2);this._drawRawCircle(t,e,i,s),t.save(),t.circle(this.x,this.y,s),t.stroke(),t.clip(),this._drawImageAtPosition(t),t.restore(),this.boundingBox.top=this.y-this.options.radius,this.boundingBox.left=this.x-this.options.radius,this.boundingBox.right=this.x+this.options.radius,this.boundingBox.bottom=this.y+this.options.radius,this._drawImageLabel(t),this.boundingBox.left=Math.min(this.boundingBox.left,this.labelDimensions.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelDimensions.left+this.labelDimensions.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelDimensions.height)},s.prototype._resizeBox=function(t){if(!this.width){var e=5,i=this.getTextSize(t);this.width=i.width+2*e,this.height=i.height+2*e,this.width+=.5*Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeWidthFactor,this.height+=.5*Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeHeightFactor,this.growthIndicator=this.width-(i.width+2*e)}},s.prototype._drawBox=function(t){this._resizeBox(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2;var e=2.5,i=this.options.borderWidth,s=this.options.borderWidthSelected||2*this.options.borderWidth;t.strokeStyle=this.selected?this.options.color.highlight.border:this.hover?this.options.color.hover.border:this.options.color.border,this.clusterSize>1&&(t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.roundRect(this.left-2*t.lineWidth,this.top-2*t.lineWidth,this.width+4*t.lineWidth,this.height+4*t.lineWidth,this.options.radius),t.stroke()),t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.fillStyle=this.selected?this.options.color.highlight.background:this.hover?this.options.color.hover.background:this.options.color.background,t.roundRect(this.left,this.top,this.width,this.height,this.options.radius),t.fill(),t.stroke(),this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height,this._label(t,this.label,this.x,this.y)},s.prototype._resizeDatabase=function(t){if(!this.width){var e=5,i=this.getTextSize(t),s=i.width+2*e;this.width=s,this.height=s,this.width+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeWidthFactor,this.height+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeHeightFactor,this.options.radius+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.width-s}},s.prototype._drawDatabase=function(t){this._resizeDatabase(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2;var e=2.5,i=this.options.borderWidth,s=this.options.borderWidthSelected||2*this.options.borderWidth;t.strokeStyle=this.selected?this.options.color.highlight.border:this.hover?this.options.color.hover.border:this.options.color.border,this.clusterSize>1&&(t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.database(this.x-this.width/2-2*t.lineWidth,this.y-.5*this.height-2*t.lineWidth,this.width+4*t.lineWidth,this.height+4*t.lineWidth),t.stroke()),t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.fillStyle=this.selected?this.options.color.highlight.background:this.hover?this.options.color.hover.background:this.options.color.background,t.database(this.x-this.width/2,this.y-.5*this.height,this.width,this.height),t.fill(),t.stroke(),this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height,this._label(t,this.label,this.x,this.y) -},s.prototype._resizeCircle=function(t){if(!this.width){var e=5,i=this.getTextSize(t),s=Math.max(i.width,i.height)+2*e;this.options.radius=s/2,this.width=s,this.height=s,this.options.radius+=.5*Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.options.radius-.5*s}},s.prototype._drawRawCircle=function(t,e,i,s){var o=2.5,n=this.options.borderWidth,r=this.options.borderWidthSelected||2*this.options.borderWidth;t.strokeStyle=this.selected?this.options.color.highlight.border:this.hover?this.options.color.hover.border:this.options.color.border,this.clusterSize>1&&(t.lineWidth=(this.selected?r:n)+(this.clusterSize>1?o:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.circle(e,i,s+2*t.lineWidth),t.stroke()),t.lineWidth=(this.selected?r:n)+(this.clusterSize>1?o:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.fillStyle=this.selected?this.options.color.highlight.background:this.hover?this.options.color.hover.background:this.options.color.background,t.circle(this.x,this.y,s),t.fill(),t.stroke()},s.prototype._drawCircle=function(t){this._resizeCircle(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2,this._drawRawCircle(t,this.x,this.y,this.options.radius),this.boundingBox.top=this.y-this.options.radius,this.boundingBox.left=this.x-this.options.radius,this.boundingBox.right=this.x+this.options.radius,this.boundingBox.bottom=this.y+this.options.radius,this._label(t,this.label,this.x,this.y)},s.prototype._resizeEllipse=function(t){if(!this.width){var e=this.getTextSize(t);this.width=1.5*e.width,this.height=2*e.height,this.width1&&(t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.ellipse(this.left-2*t.lineWidth,this.top-2*t.lineWidth,this.width+4*t.lineWidth,this.height+4*t.lineWidth),t.stroke()),t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.fillStyle=this.selected?this.options.color.highlight.background:this.hover?this.options.color.hover.background:this.options.color.background,t.ellipse(this.left,this.top,this.width,this.height),t.fill(),t.stroke(),this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height,this._label(t,this.label,this.x,this.y)},s.prototype._drawDot=function(t){this._drawShape(t,"circle")},s.prototype._drawTriangle=function(t){this._drawShape(t,"triangle")},s.prototype._drawTriangleDown=function(t){this._drawShape(t,"triangleDown")},s.prototype._drawSquare=function(t){this._drawShape(t,"square")},s.prototype._drawStar=function(t){this._drawShape(t,"star")},s.prototype._resizeShape=function(){if(!this.width){this.options.radius=this.baseRadiusValue;var t=2*this.options.radius;this.width=t,this.height=t,this.width+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeWidthFactor,this.height+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeHeightFactor,this.options.radius+=.5*Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.width-t}},s.prototype._drawShape=function(t,e){this._resizeShape(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2;var i=2.5,s=this.options.borderWidth,o=this.options.borderWidthSelected||2*this.options.borderWidth,n=2;switch(e){case"dot":n=2;break;case"square":n=2;break;case"triangle":n=3;break;case"triangleDown":n=3;break;case"star":n=4}t.strokeStyle=this.selected?this.options.color.highlight.border:this.hover?this.options.color.hover.border:this.options.color.border,this.clusterSize>1&&(t.lineWidth=(this.selected?o:s)+(this.clusterSize>1?i:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t[e](this.x,this.y,this.options.radius+n*t.lineWidth),t.stroke()),t.lineWidth=(this.selected?o:s)+(this.clusterSize>1?i:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.fillStyle=this.selected?this.options.color.highlight.background:this.hover?this.options.color.hover.background:this.options.color.background,t[e](this.x,this.y,this.options.radius),t.fill(),t.stroke(),this.boundingBox.top=this.y-this.options.radius,this.boundingBox.left=this.x-this.options.radius,this.boundingBox.right=this.x+this.options.radius,this.boundingBox.bottom=this.y+this.options.radius,this.label&&(this._label(t,this.label,this.x,this.y+this.height/2,void 0,"hanging",!0),this.boundingBox.left=Math.min(this.boundingBox.left,this.labelDimensions.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelDimensions.left+this.labelDimensions.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelDimensions.height))},s.prototype._resizeText=function(t){if(!this.width){var e=5,i=this.getTextSize(t);this.width=i.width+2*e,this.height=i.height+2*e,this.width+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeWidthFactor,this.height+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeHeightFactor,this.options.radius+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.width-(i.width+2*e)}},s.prototype._drawText=function(t){this._resizeText(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2,this._label(t,this.label,this.x,this.y),this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height},s.prototype._label=function(t,e,i,s,o,n,r){if(e&&Number(this.options.fontSize)*this.networkScale>this.fontDrawThreshold){t.font=(this.selected?"bold ":"")+this.options.fontSize+"px "+this.options.fontFace;var a=e.split("\n"),h=a.length,d=Number(this.options.fontSize),l=s+(1-h)/2*d;1==r&&(l=s+(1-h)/(2*d));for(var c=t.measureText(a[0]).width,p=1;h>p;p++){var u=t.measureText(a[p]).width;c=u>c?u:c}var m=this.options.fontSize*h,f=i-c/2,g=s-m/2;"hanging"==n&&(g+=.5*d,g+=4,l+=4),this.labelDimensions={top:g,left:f,width:c,height:m,yLine:l},void 0!==this.options.fontFill&&null!==this.options.fontFill&&"none"!==this.options.fontFill&&(t.fillStyle=this.options.fontFill,t.fillRect(f,g,c,m)),t.fillStyle=this.options.fontColor||"black",t.textAlign=o||"center",t.textBaseline=n||"middle",this.options.fontStrokeWidth>0&&(t.lineWidth=this.options.fontStrokeWidth,t.strokeStyle=this.options.fontStrokeColor,t.lineJoin="round");for(var p=0;h>p;p++)this.options.fontStrokeWidth&&t.strokeText(a[p],i,l),t.fillText(a[p],i,l),l+=d}},s.prototype.getTextSize=function(t){if(void 0!==this.label){t.font=(this.selected?"bold ":"")+this.options.fontSize+"px "+this.options.fontFace;for(var e=this.label.split("\n"),i=(Number(this.options.fontSize)+4)*e.length,s=0,o=0,n=e.length;n>o;o++)s=Math.max(s,t.measureText(e[o]).width);return{width:s,height:i,lineCount:e.length}}return{width:0,height:0,lineCount:0}},s.prototype.inArea=function(){return void 0!==this.width?this.x+this.width*this.networkScaleInv>=this.canvasTopLeft.x&&this.x-this.width*this.networkScaleInv=this.canvasTopLeft.y&&this.y-this.height*this.networkScaleInv=this.canvasTopLeft.x&&this.x=this.canvasTopLeft.y&&this.ys&&(n=s-e-this.padding),no&&(r=o-i-this.padding),ri;i++)if(e.id===r.nodes[i].id){o=r.nodes[i];break}for(o||(o={id:e.id},t.node&&(o.attr=a(o.attr,t.node))),i=n.length-1;i>=0;i--){var h=n[i];h.nodes||(h.nodes=[]),-1==h.nodes.indexOf(o)&&h.nodes.push(o)}e.attr&&(o.attr=a(o.attr,e.attr))}function l(t,e){if(t.edges||(t.edges=[]),t.edges.push(e),t.edge){var i=a({},t.edge);e.attr=a(i,e.attr)}}function c(t,e,i,s,o){var n={from:e,to:i,type:s};return t.edge&&(n.attr=a({},t.edge)),n.attr=a(n.attr||{},o),n}function p(){for(N=D.NULL,k="";" "==E||" "==E||"\n"==E||"\r"==E;)o();do{var t=!1;if("#"==E){for(var e=O-1;" "==T.charAt(e)||" "==T.charAt(e);)e--;if("\n"==T.charAt(e)||""==T.charAt(e)){for(;""!=E&&"\n"!=E;)o();t=!0}}if("/"==E&&"/"==n()){for(;""!=E&&"\n"!=E;)o();t=!0}if("/"==E&&"*"==n()){for(;""!=E;){if("*"==E&&"/"==n()){o(),o();break}o()}t=!0}for(;" "==E||" "==E||"\n"==E||"\r"==E;)o()}while(t);if(""==E)return void(N=D.DELIMITER);var i=E+n();if(C[i])return N=D.DELIMITER,k=i,o(),void o();if(C[E])return N=D.DELIMITER,k=E,void o();if(r(E)||"-"==E){for(k+=E,o();r(E);)k+=E,o();return"false"==k?k=!1:"true"==k?k=!0:isNaN(Number(k))||(k=Number(k)),void(N=D.IDENTIFIER)}if('"'==E){for(o();""!=E&&('"'!=E||'"'==E&&'"'==n());)k+=E,'"'==E&&o(),o();if('"'!=E)throw x('End of string " expected');return o(),void(N=D.IDENTIFIER)}for(N=D.UNKNOWN;""!=E;)k+=E,o();throw new SyntaxError('Syntax error in part "'+w(k,30)+'"')}function u(){var t={};if(s(),p(),"strict"==k&&(t.strict=!0,p()),("graph"==k||"digraph"==k)&&(t.type=k,p()),N==D.IDENTIFIER&&(t.id=k,p()),"{"!=k)throw x("Angle bracket { expected");if(p(),m(t),"}"!=k)throw x("Angle bracket } expected");if(p(),""!==k)throw x("End of file expected");return p(),delete t.node,delete t.edge,delete t.graph,t}function m(t){for(;""!==k&&"}"!=k;)f(t),";"==k&&p()}function f(t){var e=g(t);if(e)return void b(t,e);var i=v(t);if(!i){if(N!=D.IDENTIFIER)throw x("Identifier expected");var s=k;if(p(),"="==k){if(p(),N!=D.IDENTIFIER)throw x("Identifier expected");t[s]=k,p()}else y(t,s)}}function g(t){var e=null;if("subgraph"==k&&(e={},e.type="subgraph",p(),N==D.IDENTIFIER&&(e.id=k,p())),"{"==k){if(p(),e||(e={}),e.parent=t,e.node=t.node,e.edge=t.edge,e.graph=t.graph,m(e),"}"!=k)throw x("Angle bracket } expected");p(),delete e.node,delete e.edge,delete e.graph,delete e.parent,t.subgraphs||(t.subgraphs=[]),t.subgraphs.push(e)}return e}function v(t){return"node"==k?(p(),t.node=_(),"node"):"edge"==k?(p(),t.edge=_(),"edge"):"graph"==k?(p(),t.graph=_(),"graph"):null}function y(t,e){var i={id:e},s=_();s&&(i.attr=s),d(t,i),b(t,e)}function b(t,e){for(;"->"==k||"--"==k;){var i,s=k;p();var o=g(t);if(o)i=o;else{if(N!=D.IDENTIFIER)throw x("Identifier or subgraph expected");i=k,d(t,{id:i}),p()}var n=_(),r=c(t,e,i,s,n);l(t,r),e=i}}function _(){for(var t=null;"["==k;){for(p(),t={};""!==k&&"]"!=k;){if(N!=D.IDENTIFIER)throw x("Attribute name expected");var e=k;if(p(),"="!=k)throw x("Equal sign = expected");if(p(),N!=D.IDENTIFIER)throw x("Attribute value expected");var i=k;h(t,e,i),p(),","==k&&p()}if("]"!=k)throw x("Bracket ] expected");p()}return t}function x(t){return new SyntaxError(t+', got "'+w(k,30)+'" (char '+O+")")}function w(t,e){return t.length<=e?t:t.substr(0,27)+"..."}function S(t,e,i){Array.isArray(t)?t.forEach(function(t){Array.isArray(e)?e.forEach(function(e){i(t,e)}):i(t,e)}):Array.isArray(e)?e.forEach(function(e){i(t,e)}):i(t,e)}function M(t){var e=i(t),s={nodes:[],edges:[],options:{}};if(e.nodes&&e.nodes.forEach(function(t){var e={id:t.id,label:String(t.label||t.id)};a(e,t.attr),e.image&&(e.shape="image"),s.nodes.push(e)}),e.edges){var o=function(t){var e={from:t.from,to:t.to};return a(e,t.attr),e.style="->"==t.type?"arrow":"line",e};e.edges.forEach(function(t){var e,i;e=t.from instanceof Object?t.from.nodes:{id:t.from},i=t.to instanceof Object?t.to.nodes:{id:t.to},t.from instanceof Object&&t.from.edges&&t.from.edges.forEach(function(t){var e=o(t);s.edges.push(e)}),S(e,i,function(e,i){var n=c(s,e.id,i.id,t.type,t.attr),r=o(n);s.edges.push(r)}),t.to instanceof Object&&t.to.edges&&t.to.edges.forEach(function(t){var e=o(t);s.edges.push(e)})})}return e.attr&&(s.options=e.attr),s}var D={NULL:0,DELIMITER:1,IDENTIFIER:2,UNKNOWN:3},C={"{":!0,"}":!0,"[":!0,"]":!0,";":!0,"=":!0,",":!0,"->":!0,"--":!0},T="",O=0,E="",k="",N=D.NULL,I=/[a-zA-Z_0-9.:#]/;e.parseDOT=i,e.DOTToGraph=M},function(t,e){function i(t,e){var i=[],s=[];this.options={edges:{inheritColor:!0},nodes:{allowedToMove:!1,parseColor:!1}},void 0!==e&&(this.options.nodes.allowedToMove=e.allowedToMove|!1,this.options.nodes.parseColor=e.parseColor|!1,this.options.edges.inheritColor=e.inheritColor|!0);for(var o=t.edges,n=t.nodes,r=0;r=s&&(s=864e5),e=new Date(e.valueOf()-.05*s),i=new Date(i.valueOf()+.05*s)}return{start:e,end:i}},s.prototype.setWindow=function(t,e,i){var s=i&&void 0!==i.animate?i.animate:!0;if(1==arguments.length){var o=arguments[0];this.range.setRange(o.start,o.end,s)}else this.range.setRange(t,e,s)},s.prototype.moveTo=function(t,e){var i=this.range.end-this.range.start,s=r.convert(t,"Date").valueOf(),o=s-i/2,n=s+i/2,a=e&&void 0!==e.animate?e.animate:!0;this.range.setRange(o,n,a)},s.prototype.getWindow=function(){var t=this.range.getRange();return{start:new Date(t.start),end:new Date(t.end)}},s.prototype.redraw=function(){var t=!1,e=this.options,i=this.props,s=this.dom;if(s){h.updateHiddenDates(this.body,this.options.hiddenDates),"top"==e.orientation?(r.addClassName(s.root,"top"),r.removeClassName(s.root,"bottom")):(r.removeClassName(s.root,"top"),r.addClassName(s.root,"bottom")),s.root.style.maxHeight=r.option.asSize(e.maxHeight,""),s.root.style.minHeight=r.option.asSize(e.minHeight,""),s.root.style.width=r.option.asSize(e.width,""),i.border.left=(s.centerContainer.offsetWidth-s.centerContainer.clientWidth)/2,i.border.right=i.border.left,i.border.top=(s.centerContainer.offsetHeight-s.centerContainer.clientHeight)/2,i.border.bottom=i.border.top;var o=s.root.offsetHeight-s.root.clientHeight,n=s.root.offsetWidth-s.root.clientWidth;0===s.centerContainer.clientHeight&&(i.border.left=i.border.top,i.border.right=i.border.left),0===s.root.clientHeight&&(n=o),i.center.height=s.center.offsetHeight,i.left.height=s.left.offsetHeight,i.right.height=s.right.offsetHeight,i.top.height=s.top.clientHeight||-i.border.top,i.bottom.height=s.bottom.clientHeight||-i.border.bottom;var a=Math.max(i.left.height,i.center.height,i.right.height),d=i.top.height+a+i.bottom.height+o+i.border.top+i.border.bottom;s.root.style.height=r.option.asSize(e.height,d+"px"),i.root.height=s.root.offsetHeight,i.background.height=i.root.height-o;var l=i.root.height-i.top.height-i.bottom.height-o;i.centerContainer.height=l,i.leftContainer.height=l,i.rightContainer.height=i.leftContainer.height,i.root.width=s.root.offsetWidth,i.background.width=i.root.width-n,i.left.width=s.leftContainer.clientWidth||-i.border.left,i.leftContainer.width=i.left.width,i.right.width=s.rightContainer.clientWidth||-i.border.right,i.rightContainer.width=i.right.width;var c=i.root.width-i.left.width-i.right.width-n;i.center.width=c,i.centerContainer.width=c,i.top.width=c,i.bottom.width=c,s.background.style.height=i.background.height+"px",s.backgroundVertical.style.height=i.background.height+"px",s.backgroundHorizontal.style.height=i.centerContainer.height+"px",s.centerContainer.style.height=i.centerContainer.height+"px",s.leftContainer.style.height=i.leftContainer.height+"px",s.rightContainer.style.height=i.rightContainer.height+"px",s.background.style.width=i.background.width+"px",s.backgroundVertical.style.width=i.centerContainer.width+"px",s.backgroundHorizontal.style.width=i.background.width+"px",s.centerContainer.style.width=i.center.width+"px",s.top.style.width=i.top.width+"px",s.bottom.style.width=i.bottom.width+"px",s.background.style.left="0",s.background.style.top="0",s.backgroundVertical.style.left=i.left.width+i.border.left+"px",s.backgroundVertical.style.top="0",s.backgroundHorizontal.style.left="0",s.backgroundHorizontal.style.top=i.top.height+"px",s.centerContainer.style.left=i.left.width+"px",s.centerContainer.style.top=i.top.height+"px",s.leftContainer.style.left="0",s.leftContainer.style.top=i.top.height+"px",s.rightContainer.style.left=i.left.width+i.center.width+"px",s.rightContainer.style.top=i.top.height+"px",s.top.style.left=i.left.width+"px",s.top.style.top="0",s.bottom.style.left=i.left.width+"px",s.bottom.style.top=i.top.height+i.centerContainer.height+"px",this._updateScrollTop();var p=this.props.scrollTop;"bottom"==e.orientation&&(p+=Math.max(this.props.centerContainer.height-this.props.center.height-this.props.border.top-this.props.border.bottom,0)),s.center.style.left="0",s.center.style.top=p+"px",s.left.style.left="0",s.left.style.top=p+"px",s.right.style.left="0",s.right.style.top=p+"px";var u=0==this.props.scrollTop?"hidden":"",m=this.props.scrollTop==this.props.scrollTopMin?"hidden":"";if(s.shadowTop.style.visibility=u,s.shadowBottom.style.visibility=m,s.shadowTopLeft.style.visibility=u,s.shadowBottomLeft.style.visibility=m,s.shadowTopRight.style.visibility=u,s.shadowBottomRight.style.visibility=m,this.components.forEach(function(e){t=e.redraw()||t}),t){var f=3;this.redrawCount0&&(this.props.scrollTop=0),this.props.scrollTops;s++){var o=s%2===0?1.3*i:.5*i;this.lineTo(t+o*Math.sin(2*s*Math.PI/10),e-o*Math.cos(2*s*Math.PI/10))}this.closePath()},CanvasRenderingContext2D.prototype.roundRect=function(t,e,i,s,o){var n=Math.PI/180;0>i-2*o&&(o=i/2),0>s-2*o&&(o=s/2),this.beginPath(),this.moveTo(t+o,e),this.lineTo(t+i-o,e),this.arc(t+i-o,e+o,o,270*n,360*n,!1),this.lineTo(t+i,e+s-o),this.arc(t+i-o,e+s-o,o,0,90*n,!1),this.lineTo(t+o,e+s),this.arc(t+o,e+s-o,o,90*n,180*n,!1),this.lineTo(t,e+o),this.arc(t+o,e+o,o,180*n,270*n,!1)},CanvasRenderingContext2D.prototype.ellipse=function(t,e,i,s){var o=.5522848,n=i/2*o,r=s/2*o,a=t+i,h=e+s,d=t+i/2,l=e+s/2; -this.beginPath(),this.moveTo(t,l),this.bezierCurveTo(t,l-r,d-n,e,d,e),this.bezierCurveTo(d+n,e,a,l-r,a,l),this.bezierCurveTo(a,l+r,d+n,h,d,h),this.bezierCurveTo(d-n,h,t,l+r,t,l)},CanvasRenderingContext2D.prototype.database=function(t,e,i,s){var o=1/3,n=i,r=s*o,a=.5522848,h=n/2*a,d=r/2*a,l=t+n,c=e+r,p=t+n/2,u=e+r/2,m=e+(s-r/2),f=e+s;this.beginPath(),this.moveTo(l,u),this.bezierCurveTo(l,u+d,p+h,c,p,c),this.bezierCurveTo(p-h,c,t,u+d,t,u),this.bezierCurveTo(t,u-d,p-h,e,p,e),this.bezierCurveTo(p+h,e,l,u-d,l,u),this.lineTo(l,m),this.bezierCurveTo(l,m+d,p+h,f,p,f),this.bezierCurveTo(p-h,f,t,m+d,t,m),this.lineTo(t,u)},CanvasRenderingContext2D.prototype.arrow=function(t,e,i,s){var o=t-s*Math.cos(i),n=e-s*Math.sin(i),r=t-.9*s*Math.cos(i),a=e-.9*s*Math.sin(i),h=o+s/3*Math.cos(i+.5*Math.PI),d=n+s/3*Math.sin(i+.5*Math.PI),l=o+s/3*Math.cos(i-.5*Math.PI),c=n+s/3*Math.sin(i-.5*Math.PI);this.beginPath(),this.moveTo(t,e),this.lineTo(h,d),this.lineTo(r,a),this.lineTo(l,c),this.closePath()},CanvasRenderingContext2D.prototype.dashedLine=function(t,e,i,s,o){o||(o=[10,5]),0==p&&(p=.001);var n=o.length;this.moveTo(t,e);for(var r=i-t,a=s-e,h=a/r,d=Math.sqrt(r*r+a*a),l=0,c=!0;d>=.1;){var p=o[l++%n];p>d&&(p=d);var u=Math.sqrt(p*p/(1+h*h));0>r&&(u=-u),t+=u,e+=h*u,this[c?"lineTo":"moveTo"](t,e),d-=p,c=!c}})},function(t,e,i){function s(t,e){this.groupId=t,this.options=e}var o=i(2),n=i(53);s.prototype.getYRange=function(t){for(var e=t[0].y,i=t[0].y,s=0;st[s].y?t[s].y:e,i=i0){var r,a,h=Number(i.svg.style.height.replace("px",""));if(r=o.getSVGElement("path",i.svgElements,i.svg),r.setAttributeNS(null,"class",e.className),void 0!==e.style&&r.setAttributeNS(null,"style",e.style),a=1==e.options.catmullRom.enabled?s._catmullRom(t,e):s._linear(t),1==e.options.shaded.enabled){var d,l=o.getSVGElement("path",i.svgElements,i.svg);d="top"==e.options.shaded.orientation?"M"+t[0].x+",0 "+a+"L"+t[t.length-1].x+",0":"M"+t[0].x+","+h+" "+a+"L"+t[t.length-1].x+","+h,l.setAttributeNS(null,"class",e.className+" fill"),void 0!==e.options.shaded.style&&l.setAttributeNS(null,"style",e.options.shaded.style),l.setAttributeNS(null,"d",d)}r.setAttributeNS(null,"d","M"+a),1==e.options.drawPoints.enabled&&n.draw(t,e,i)}},s._catmullRomUniform=function(t){for(var e,i,s,o,n,r,a=Math.round(t[0].x)+","+Math.round(t[0].y)+" ",h=1/6,d=t.length,l=0;d-1>l;l++)e=0==l?t[0]:t[l-1],i=t[l],s=t[l+1],o=d>l+2?t[l+2]:s,n={x:(-e.x+6*i.x+s.x)*h,y:(-e.y+6*i.y+s.y)*h},r={x:(i.x+6*s.x-o.x)*h,y:(i.y+6*s.y-o.y)*h},a+="C"+n.x+","+n.y+" "+r.x+","+r.y+" "+s.x+","+s.y+" ";return a},s._catmullRom=function(t,e){var i=e.options.catmullRom.alpha;if(0==i||void 0===i)return this._catmullRomUniform(t);for(var s,o,n,r,a,h,d,l,c,p,u,m,f,g,v,y,b,_,x,w=Math.round(t[0].x)+","+Math.round(t[0].y)+" ",S=t.length,M=0;S-1>M;M++)s=0==M?t[0]:t[M-1],o=t[M],n=t[M+1],r=S>M+2?t[M+2]:n,d=Math.sqrt(Math.pow(s.x-o.x,2)+Math.pow(s.y-o.y,2)),l=Math.sqrt(Math.pow(o.x-n.x,2)+Math.pow(o.y-n.y,2)),c=Math.sqrt(Math.pow(n.x-r.x,2)+Math.pow(n.y-r.y,2)),g=Math.pow(c,i),y=Math.pow(c,2*i),v=Math.pow(l,i),b=Math.pow(l,2*i),x=Math.pow(d,i),_=Math.pow(d,2*i),p=2*_+3*x*v+b,u=2*y+3*g*v+b,m=3*x*(x+v),m>0&&(m=1/m),f=3*g*(g+v),f>0&&(f=1/f),a={x:(-b*s.x+p*o.x+_*n.x)*m,y:(-b*s.y+p*o.y+_*n.y)*m},h={x:(y*o.x+u*n.x-b*r.x)*f,y:(y*o.y+u*n.y-b*r.y)*f},0==a.x&&0==a.y&&(a=o),0==h.x&&0==h.y&&(h=n),w+="C"+a.x+","+a.y+" "+h.x+","+h.y+" "+n.x+","+n.y+" ";return w},s._linear=function(t){for(var e="",i=0;it[s].y?t[s].y:e,i=i0&&(n=Math.min(n,Math.abs(c[d-1].x-r))),a=s._getSafeDrawData(n,h,m);else{var g=d+(p[r].amount-p[r].resolved),v=d-(p[r].resolved+1);g0&&(n=Math.min(n,Math.abs(c[v].x-r))),a=s._getSafeDrawData(n,h,m),p[r].resolved+=1,"stack"==h.options.barChart.handleOverlap?(f=p[r].accumulated,p[r].accumulated+=h.zeroPosition-c[d].y):"sideBySide"==h.options.barChart.handleOverlap&&(a.width=a.width/p[r].amount,a.offset+=p[r].resolved*a.width-.5*a.width*(p[r].amount+1),"left"==h.options.barChart.align?a.offset-=.5*a.width:"right"==h.options.barChart.align&&(a.offset+=.5*a.width))}o.drawBar(c[d].x+a.offset,c[d].y-f,a.width,h.zeroPosition-c[d].y,h.className+" bar",i.svgElements,i.svg),1==h.options.drawPoints.enabled&&o.drawPoint(c[d].x+a.offset,c[d].y,h,i.svgElements,i.svg)}},s._getDataIntersections=function(t,e){for(var i,s=0;s0&&(i=Math.min(i,Math.abs(e[s-1].x-e[s].x))),0==i&&(void 0===t[e[s].x]&&(t[e[s].x]={amount:0,resolved:0,accumulated:0}),t[e[s].x].amount+=1)},s._getSafeDrawData=function(t,e,i){var s,o;return t0?(s=i>t?i:t,o=0,"left"==e.options.barChart.align?o-=.5*t:"right"==e.options.barChart.align&&(o+=.5*t)):(s=e.options.barChart.width,o=0,"left"==e.options.barChart.align?o-=.5*e.options.barChart.width:"right"==e.options.barChart.align&&(o+=.5*e.options.barChart.width)),{width:s,offset:o}},s.getStackedBarYRange=function(t,e,i,o,n){if(t.length>0){t.sort(function(t,e){return t.x==e.x?t.groupId-e.groupId:t.x-e.x});var r={};s._getDataIntersections(r,t),e[o]=s._getStackedBarYRange(r,t),e[o].yAxisOrientation=n,i.push(o)}},s._getStackedBarYRange=function(t,e){for(var i,s=e[0].y,o=e[0].y,n=0;ne[n].y?e[n].y:s,o=ot[r].accumulated?t[r].accumulated:s,o=ot[s].y?t[s].y:e,i=is;++s)i[s].apply(this,e)}return this},e.prototype.listeners=function(t){return this._callbacks=this._callbacks||{},this._callbacks[t]||[]},e.prototype.hasListeners=function(t){return!!this.listeners(t).length}},function(t,e,i){var s;(function(t,o){(function(n){function r(t,e,i){switch(arguments.length){case 2:return null!=t?t:e;case 3:return null!=t?t:null!=e?e:i;default:throw new Error("Implement me")}}function a(t,e){return Ie.call(t,e)}function h(){return{empty:!1,unusedTokens:[],unusedInput:[],overflow:-2,charsLeftOver:0,nullInput:!1,invalidMonth:null,invalidFormat:!1,userInvalidated:!1,iso:!1}}function d(t){Ce.suppressDeprecationWarnings===!1&&"undefined"!=typeof console&&console.warn&&console.warn("Deprecation warning: "+t)}function l(t,e){var i=!0;return b(function(){return i&&(d(t),i=!1),e.apply(this,arguments)},e)}function c(t,e){Si[t]||(d(e),Si[t]=!0)}function p(t,e){return function(i){return w(t.call(this,i),e)}}function u(t,e){return function(i){return this.localeData().ordinal(t.call(this,i),e)}}function m(t,e){var i,s,o=12*(e.year()-t.year())+(e.month()-t.month()),n=t.clone().add(o,"months");return 0>e-n?(i=t.clone().add(o-1,"months"),s=(e-n)/(n-i)):(i=t.clone().add(o+1,"months"),s=(e-n)/(i-n)),-(o+s)}function f(t,e,i){var s;return null==i?e:null!=t.meridiemHour?t.meridiemHour(e,i):null!=t.isPM?(s=t.isPM(i),s&&12>e&&(e+=12),s||12!==e||(e=0),e):e}function g(){}function v(t,e){e!==!1&&F(t),_(this,t),this._d=new Date(+t._d),Di===!1&&(Di=!0,Ce.updateOffset(this),Di=!1)}function y(t){var e=N(t),i=e.year||0,s=e.quarter||0,o=e.month||0,n=e.week||0,r=e.day||0,a=e.hour||0,h=e.minute||0,d=e.second||0,l=e.millisecond||0;this._milliseconds=+l+1e3*d+6e4*h+36e5*a,this._days=+r+7*n,this._months=+o+3*s+12*i,this._data={},this._locale=Ce.localeData(),this._bubble()}function b(t,e){for(var i in e)a(e,i)&&(t[i]=e[i]);return a(e,"toString")&&(t.toString=e.toString),a(e,"valueOf")&&(t.valueOf=e.valueOf),t}function _(t,e){var i,s,o;if("undefined"!=typeof e._isAMomentObject&&(t._isAMomentObject=e._isAMomentObject),"undefined"!=typeof e._i&&(t._i=e._i),"undefined"!=typeof e._f&&(t._f=e._f),"undefined"!=typeof e._l&&(t._l=e._l),"undefined"!=typeof e._strict&&(t._strict=e._strict),"undefined"!=typeof e._tzm&&(t._tzm=e._tzm),"undefined"!=typeof e._isUTC&&(t._isUTC=e._isUTC),"undefined"!=typeof e._offset&&(t._offset=e._offset),"undefined"!=typeof e._pf&&(t._pf=e._pf),"undefined"!=typeof e._locale&&(t._locale=e._locale),Ye.length>0)for(i in Ye)s=Ye[i],o=e[s],"undefined"!=typeof o&&(t[s]=o);return t}function x(t){return 0>t?Math.ceil(t):Math.floor(t)}function w(t,e,i){for(var s=""+Math.abs(t),o=t>=0;s.lengths;s++)(i&&t[s]!==e[s]||!i&&L(t[s])!==L(e[s]))&&r++;return r+n}function k(t){if(t){var e=t.toLowerCase().replace(/(.)s$/,"$1");t=gi[t]||vi[e]||e}return t}function N(t){var e,i,s={};for(i in t)a(t,i)&&(e=k(i),e&&(s[e]=t[i]));return s}function I(t){var e,i;if(0===t.indexOf("week"))e=7,i="day";else{if(0!==t.indexOf("month"))return;e=12,i="month"}Ce[t]=function(s,o){var r,a,h=Ce._locale[t],d=[];if("number"==typeof s&&(o=s,s=n),a=function(t){var e=Ce().utc().set(i,t);return h.call(Ce._locale,e,s||"")},null!=o)return a(o);for(r=0;e>r;r++)d.push(a(r));return d}}function L(t){var e=+t,i=0;return 0!==e&&isFinite(e)&&(i=e>=0?Math.floor(e):Math.ceil(e)),i}function z(t,e){return new Date(Date.UTC(t,e+1,0)).getUTCDate()}function P(t,e,i){return me(Ce([t,11,31+e-i]),e,i).week}function A(t){return R(t)?366:365}function R(t){return t%4===0&&t%100!==0||t%400===0}function F(t){var e;t._a&&-2===t._pf.overflow&&(e=t._a[ze]<0||t._a[ze]>11?ze:t._a[Pe]<1||t._a[Pe]>z(t._a[Le],t._a[ze])?Pe:t._a[Ae]<0||t._a[Ae]>24||24===t._a[Ae]&&(0!==t._a[Re]||0!==t._a[Fe]||0!==t._a[He])?Ae:t._a[Re]<0||t._a[Re]>59?Re:t._a[Fe]<0||t._a[Fe]>59?Fe:t._a[He]<0||t._a[He]>999?He:-1,t._pf._overflowDayOfYear&&(Le>e||e>Pe)&&(e=Pe),t._pf.overflow=e)}function H(t){return null==t._isValid&&(t._isValid=!isNaN(t._d.getTime())&&t._pf.overflow<0&&!t._pf.empty&&!t._pf.invalidMonth&&!t._pf.nullInput&&!t._pf.invalidFormat&&!t._pf.userInvalidated,t._strict&&(t._isValid=t._isValid&&0===t._pf.charsLeftOver&&0===t._pf.unusedTokens.length&&t._pf.bigHour===n)),t._isValid}function B(t){return t?t.toLowerCase().replace("_","-"):t}function Y(t){for(var e,i,s,o,n=0;n0;){if(s=W(o.slice(0,e).join("-")))return s;if(i&&i.length>=e&&E(o,i,!0)>=e-1)break;e--}n++}return null}function W(t){var e=null;if(!Be[t]&&We)try{e=Ce.locale(),!function(){var t=new Error('Cannot find module "./locale"');throw t.code="MODULE_NOT_FOUND",t}(),Ce.locale(e)}catch(i){}return Be[t]}function G(t,e){var i,s;return e._isUTC?(i=e.clone(),s=(Ce.isMoment(t)||O(t)?+t:+Ce(t))-+i,i._d.setTime(+i._d+s),Ce.updateOffset(i,!1),i):Ce(t).local()}function j(t){return t.match(/\[[\s\S]/)?t.replace(/^\[|\]$/g,""):t.replace(/\\/g,"")}function U(t){var e,i,s=t.match(Ve);for(e=0,i=s.length;i>e;e++)s[e]=wi[s[e]]?wi[s[e]]:j(s[e]);return function(o){var n="";for(e=0;i>e;e++)n+=s[e]instanceof Function?s[e].call(o,t):s[e];return n}}function V(t,e){return t.isValid()?(e=X(e,t.localeData()),yi[e]||(yi[e]=U(e)),yi[e](t)):t.localeData().invalidDate()}function X(t,e){function i(t){return e.longDateFormat(t)||t}var s=5;for(Xe.lastIndex=0;s>=0&&Xe.test(t);)t=t.replace(Xe,i),Xe.lastIndex=0,s-=1;return t}function q(t,e){var i,s=e._strict;switch(t){case"Q":return oi;case"DDDD":return ri;case"YYYY":case"GGGG":case"gggg":return s?ai:Qe;case"Y":case"G":case"g":return di;case"YYYYYY":case"YYYYY":case"GGGGG":case"ggggg":return s?hi:Ke;case"S":if(s)return oi;case"SS":if(s)return ni;case"SSS":if(s)return ri;case"DDD":return Ze;case"MMM":case"MMMM":case"dd":case"ddd":case"dddd":return Je;case"a":case"A":return e._locale._meridiemParse;case"x":return ii;case"X":return si;case"Z":case"ZZ":return ti;case"T":return ei;case"SSSS":return $e;case"MM":case"DD":case"YY":case"GG":case"gg":case"HH":case"hh":case"mm":case"ss":case"ww":case"WW":return s?ni:qe;case"M":case"D":case"d":case"H":case"h":case"m":case"s":case"w":case"W":case"e":case"E":return qe;case"Do":return s?e._locale._ordinalParse:e._locale._ordinalParseLenient;default:return i=new RegExp(se(ie(t.replace("\\","")),"i"))}}function Z(t){t=t||"";var e=t.match(ti)||[],i=e[e.length-1]||[],s=(i+"").match(mi)||["-",0,0],o=+(60*s[1])+L(s[2]);return"+"===s[0]?o:-o}function Q(t,e,i){var s,o=i._a;switch(t){case"Q":null!=e&&(o[ze]=3*(L(e)-1));break;case"M":case"MM":null!=e&&(o[ze]=L(e)-1);break;case"MMM":case"MMMM":s=i._locale.monthsParse(e,t,i._strict),null!=s?o[ze]=s:i._pf.invalidMonth=e;break;case"D":case"DD":null!=e&&(o[Pe]=L(e));break;case"Do":null!=e&&(o[Pe]=L(parseInt(e.match(/\d{1,2}/)[0],10)));break;case"DDD":case"DDDD":null!=e&&(i._dayOfYear=L(e));break;case"YY":o[Le]=Ce.parseTwoDigitYear(e);break;case"YYYY":case"YYYYY":case"YYYYYY":o[Le]=L(e);break;case"a":case"A":i._meridiem=e;break;case"h":case"hh":i._pf.bigHour=!0;case"H":case"HH":o[Ae]=L(e);break;case"m":case"mm":o[Re]=L(e);break;case"s":case"ss":o[Fe]=L(e);break;case"S":case"SS":case"SSS":case"SSSS":o[He]=L(1e3*("0."+e));break;case"x":i._d=new Date(L(e));break;case"X":i._d=new Date(1e3*parseFloat(e));break;case"Z":case"ZZ":i._useUTC=!0,i._tzm=Z(e);break;case"dd":case"ddd":case"dddd":s=i._locale.weekdaysParse(e),null!=s?(i._w=i._w||{},i._w.d=s):i._pf.invalidWeekday=e;break;case"w":case"ww":case"W":case"WW":case"d":case"e":case"E":t=t.substr(0,1);case"gggg":case"GGGG":case"GGGGG":t=t.substr(0,2),e&&(i._w=i._w||{},i._w[t]=L(e));break;case"gg":case"GG":i._w=i._w||{},i._w[t]=Ce.parseTwoDigitYear(e)}}function K(t){var e,i,s,o,n,a,h;e=t._w,null!=e.GG||null!=e.W||null!=e.E?(n=1,a=4,i=r(e.GG,t._a[Le],me(Ce(),1,4).year),s=r(e.W,1),o=r(e.E,1)):(n=t._locale._week.dow,a=t._locale._week.doy,i=r(e.gg,t._a[Le],me(Ce(),n,a).year),s=r(e.w,1),null!=e.d?(o=e.d,n>o&&++s):o=null!=e.e?e.e+n:n),h=fe(i,s,o,a,n),t._a[Le]=h.year,t._dayOfYear=h.dayOfYear}function $(t){var e,i,s,o,n=[];if(!t._d){for(s=te(t),t._w&&null==t._a[Pe]&&null==t._a[ze]&&K(t),t._dayOfYear&&(o=r(t._a[Le],s[Le]),t._dayOfYear>A(o)&&(t._pf._overflowDayOfYear=!0),i=le(o,0,t._dayOfYear),t._a[ze]=i.getUTCMonth(),t._a[Pe]=i.getUTCDate()),e=0;3>e&&null==t._a[e];++e)t._a[e]=n[e]=s[e];for(;7>e;e++)t._a[e]=n[e]=null==t._a[e]?2===e?1:0:t._a[e];24===t._a[Ae]&&0===t._a[Re]&&0===t._a[Fe]&&0===t._a[He]&&(t._nextDay=!0,t._a[Ae]=0),t._d=(t._useUTC?le:de).apply(null,n),null!=t._tzm&&t._d.setUTCMinutes(t._d.getUTCMinutes()-t._tzm),t._nextDay&&(t._a[Ae]=24)}}function J(t){var e;t._d||(e=N(t._i),t._a=[e.year,e.month,e.day||e.date,e.hour,e.minute,e.second,e.millisecond],$(t))}function te(t){var e=new Date;return t._useUTC?[e.getUTCFullYear(),e.getUTCMonth(),e.getUTCDate()]:[e.getFullYear(),e.getMonth(),e.getDate()]}function ee(t){if(t._f===Ce.ISO_8601)return void ne(t);t._a=[],t._pf.empty=!0;var e,i,s,o,r,a=""+t._i,h=a.length,d=0;for(s=X(t._f,t._locale).match(Ve)||[],e=0;e0&&t._pf.unusedInput.push(r),a=a.slice(a.indexOf(i)+i.length),d+=i.length),wi[o]?(i?t._pf.empty=!1:t._pf.unusedTokens.push(o),Q(o,i,t)):t._strict&&!i&&t._pf.unusedTokens.push(o);t._pf.charsLeftOver=h-d,a.length>0&&t._pf.unusedInput.push(a),t._pf.bigHour===!0&&t._a[Ae]<=12&&(t._pf.bigHour=n),t._a[Ae]=f(t._locale,t._a[Ae],t._meridiem),$(t),F(t)}function ie(t){return t.replace(/\\(\[)|\\(\])|\[([^\]\[]*)\]|\\(.)/g,function(t,e,i,s,o){return e||i||s||o})}function se(t){return t.replace(/[-\/\\^$*+?.()|[\]{}]/g,"\\$&")}function oe(t){var e,i,s,o,n;if(0===t._f.length)return t._pf.invalidFormat=!0,void(t._d=new Date(0/0));for(o=0;on)&&(s=n,i=e));b(t,i||e)}function ne(t){var e,i,s=t._i,o=li.exec(s);if(o){for(t._pf.iso=!0,e=0,i=pi.length;i>e;e++)if(pi[e][1].exec(s)){t._f=pi[e][0]+(o[6]||" ");break}for(e=0,i=ui.length;i>e;e++)if(ui[e][1].exec(s)){t._f+=ui[e][0];break}s.match(ti)&&(t._f+="Z"),ee(t)}else t._isValid=!1}function re(t){ne(t),t._isValid===!1&&(delete t._isValid,Ce.createFromInputFallback(t))}function ae(t,e){var i,s=[];for(i=0;it&&a.setFullYear(t),a}function le(t){var e=new Date(Date.UTC.apply(null,arguments));return 1970>t&&e.setUTCFullYear(t),e}function ce(t,e){if("string"==typeof t)if(isNaN(t)){if(t=e.weekdaysParse(t),"number"!=typeof t)return null}else t=parseInt(t,10);return t}function pe(t,e,i,s,o){return o.relativeTime(e||1,!!i,t,s)}function ue(t,e,i){var s=Ce.duration(t).abs(),o=Ne(s.as("s")),n=Ne(s.as("m")),r=Ne(s.as("h")),a=Ne(s.as("d")),h=Ne(s.as("M")),d=Ne(s.as("y")),l=o0,l[4]=i,pe.apply({},l)}function me(t,e,i){var s,o=i-e,n=i-t.day();return n>o&&(n-=7),o-7>n&&(n+=7),s=Ce(t).add(n,"d"),{week:Math.ceil(s.dayOfYear()/7),year:s.year()}}function fe(t,e,i,s,o){var n,r,a=le(t,0,1).getUTCDay();return a=0===a?7:a,i=null!=i?i:o,n=o-a+(a>s?7:0)-(o>a?7:0),r=7*(e-1)+(i-o)+n+1,{year:r>0?t:t-1,dayOfYear:r>0?r:A(t-1)+r}}function ge(t){var e,i=t._i,s=t._f;return t._locale=t._locale||Ce.localeData(t._l),null===i||s===n&&""===i?Ce.invalid({nullInput:!0}):("string"==typeof i&&(t._i=i=t._locale.preparse(i)),Ce.isMoment(i)?new v(i,!0):(s?T(s)?oe(t):ee(t):he(t),e=new v(t),e._nextDay&&(e.add(1,"d"),e._nextDay=n),e))}function ve(t,e){var i,s;if(1===e.length&&T(e[0])&&(e=e[0]),!e.length)return Ce();for(i=e[0],s=1;s=0?"+":"-";return e+w(Math.abs(t),6)},gg:function(){return w(this.weekYear()%100,2)},gggg:function(){return w(this.weekYear(),4)},ggggg:function(){return w(this.weekYear(),5)},GG:function(){return w(this.isoWeekYear()%100,2)},GGGG:function(){return w(this.isoWeekYear(),4)},GGGGG:function(){return w(this.isoWeekYear(),5)},e:function(){return this.weekday()},E:function(){return this.isoWeekday()},a:function(){return this.localeData().meridiem(this.hours(),this.minutes(),!0)},A:function(){return this.localeData().meridiem(this.hours(),this.minutes(),!1)},H:function(){return this.hours()},h:function(){return this.hours()%12||12},m:function(){return this.minutes()},s:function(){return this.seconds()},S:function(){return L(this.milliseconds()/100)},SS:function(){return w(L(this.milliseconds()/10),2)},SSS:function(){return w(this.milliseconds(),3)},SSSS:function(){return w(this.milliseconds(),3)},Z:function(){var t=this.utcOffset(),e="+";return 0>t&&(t=-t,e="-"),e+w(L(t/60),2)+":"+w(L(t)%60,2)},ZZ:function(){var t=this.utcOffset(),e="+";return 0>t&&(t=-t,e="-"),e+w(L(t/60),2)+w(L(t)%60,2)},z:function(){return this.zoneAbbr()},zz:function(){return this.zoneName()},x:function(){return this.valueOf()},X:function(){return this.unix()},Q:function(){return this.quarter()}},Si={},Mi=["months","monthsShort","weekdays","weekdaysShort","weekdaysMin"],Di=!1;_i.length;)Oe=_i.pop(),wi[Oe+"o"]=u(wi[Oe],Oe);for(;xi.length;)Oe=xi.pop(),wi[Oe+Oe]=p(wi[Oe],2);wi.DDDD=p(wi.DDD,3),b(g.prototype,{set:function(t){var e,i;for(i in t)e=t[i],"function"==typeof e?this[i]=e:this["_"+i]=e;this._ordinalParseLenient=new RegExp(this._ordinalParse.source+"|"+/\d{1,2}/.source)},_months:"January_February_March_April_May_June_July_August_September_October_November_December".split("_"),months:function(t){return this._months[t.month()]},_monthsShort:"Jan_Feb_Mar_Apr_May_Jun_Jul_Aug_Sep_Oct_Nov_Dec".split("_"),monthsShort:function(t){return this._monthsShort[t.month()]},monthsParse:function(t,e,i){var s,o,n;for(this._monthsParse||(this._monthsParse=[],this._longMonthsParse=[],this._shortMonthsParse=[]),s=0;12>s;s++){if(o=Ce.utc([2e3,s]),i&&!this._longMonthsParse[s]&&(this._longMonthsParse[s]=new RegExp("^"+this.months(o,"").replace(".","")+"$","i"),this._shortMonthsParse[s]=new RegExp("^"+this.monthsShort(o,"").replace(".","")+"$","i")),i||this._monthsParse[s]||(n="^"+this.months(o,"")+"|^"+this.monthsShort(o,""),this._monthsParse[s]=new RegExp(n.replace(".",""),"i")),i&&"MMMM"===e&&this._longMonthsParse[s].test(t))return s;if(i&&"MMM"===e&&this._shortMonthsParse[s].test(t))return s;if(!i&&this._monthsParse[s].test(t))return s}},_weekdays:"Sunday_Monday_Tuesday_Wednesday_Thursday_Friday_Saturday".split("_"),weekdays:function(t){return this._weekdays[t.day()]},_weekdaysShort:"Sun_Mon_Tue_Wed_Thu_Fri_Sat".split("_"),weekdaysShort:function(t){return this._weekdaysShort[t.day()]},_weekdaysMin:"Su_Mo_Tu_We_Th_Fr_Sa".split("_"),weekdaysMin:function(t){return this._weekdaysMin[t.day()]},weekdaysParse:function(t){var e,i,s;for(this._weekdaysParse||(this._weekdaysParse=[]),e=0;7>e;e++)if(this._weekdaysParse[e]||(i=Ce([2e3,1]).day(e),s="^"+this.weekdays(i,"")+"|^"+this.weekdaysShort(i,"")+"|^"+this.weekdaysMin(i,""),this._weekdaysParse[e]=new RegExp(s.replace(".",""),"i")),this._weekdaysParse[e].test(t))return e},_longDateFormat:{LTS:"h:mm:ss A",LT:"h:mm A",L:"MM/DD/YYYY",LL:"MMMM D, YYYY",LLL:"MMMM D, YYYY LT",LLLL:"dddd, MMMM D, YYYY LT"},longDateFormat:function(t){var e=this._longDateFormat[t]; -return!e&&this._longDateFormat[t.toUpperCase()]&&(e=this._longDateFormat[t.toUpperCase()].replace(/MMMM|MM|DD|dddd/g,function(t){return t.slice(1)}),this._longDateFormat[t]=e),e},isPM:function(t){return"p"===(t+"").toLowerCase().charAt(0)},_meridiemParse:/[ap]\.?m?\.?/i,meridiem:function(t,e,i){return t>11?i?"pm":"PM":i?"am":"AM"},_calendar:{sameDay:"[Today at] LT",nextDay:"[Tomorrow at] LT",nextWeek:"dddd [at] LT",lastDay:"[Yesterday at] LT",lastWeek:"[Last] dddd [at] LT",sameElse:"L"},calendar:function(t,e,i){var s=this._calendar[t];return"function"==typeof s?s.apply(e,[i]):s},_relativeTime:{future:"in %s",past:"%s ago",s:"a few seconds",m:"a minute",mm:"%d minutes",h:"an hour",hh:"%d hours",d:"a day",dd:"%d days",M:"a month",MM:"%d months",y:"a year",yy:"%d years"},relativeTime:function(t,e,i,s){var o=this._relativeTime[i];return"function"==typeof o?o(t,e,i,s):o.replace(/%d/i,t)},pastFuture:function(t,e){var i=this._relativeTime[t>0?"future":"past"];return"function"==typeof i?i(e):i.replace(/%s/i,e)},ordinal:function(t){return this._ordinal.replace("%d",t)},_ordinal:"%d",_ordinalParse:/\d{1,2}/,preparse:function(t){return t},postformat:function(t){return t},week:function(t){return me(t,this._week.dow,this._week.doy).week},_week:{dow:0,doy:6},firstDayOfWeek:function(){return this._week.dow},firstDayOfYear:function(){return this._week.doy},_invalidDate:"Invalid date",invalidDate:function(){return this._invalidDate}}),Ce=function(t,e,i,s){var o;return"boolean"==typeof i&&(s=i,i=n),o={},o._isAMomentObject=!0,o._i=t,o._f=e,o._l=i,o._strict=s,o._isUTC=!1,o._pf=h(),ge(o)},Ce.suppressDeprecationWarnings=!1,Ce.createFromInputFallback=l("moment construction falls back to js Date. This is discouraged and will be removed in upcoming major release. Please refer to https://github.com/moment/moment/issues/1407 for more info.",function(t){t._d=new Date(t._i+(t._useUTC?" UTC":""))}),Ce.min=function(){var t=[].slice.call(arguments,0);return ve("isBefore",t)},Ce.max=function(){var t=[].slice.call(arguments,0);return ve("isAfter",t)},Ce.utc=function(t,e,i,s){var o;return"boolean"==typeof i&&(s=i,i=n),o={},o._isAMomentObject=!0,o._useUTC=!0,o._isUTC=!0,o._l=i,o._i=t,o._f=e,o._strict=s,o._pf=h(),ge(o).utc()},Ce.unix=function(t){return Ce(1e3*t)},Ce.duration=function(t,e){var i,s,o,n,r=t,h=null;return Ce.isDuration(t)?r={ms:t._milliseconds,d:t._days,M:t._months}:"number"==typeof t?(r={},e?r[e]=t:r.milliseconds=t):(h=je.exec(t))?(i="-"===h[1]?-1:1,r={y:0,d:L(h[Pe])*i,h:L(h[Ae])*i,m:L(h[Re])*i,s:L(h[Fe])*i,ms:L(h[He])*i}):(h=Ue.exec(t))?(i="-"===h[1]?-1:1,o=function(t){var e=t&&parseFloat(t.replace(",","."));return(isNaN(e)?0:e)*i},r={y:o(h[2]),M:o(h[3]),d:o(h[4]),h:o(h[5]),m:o(h[6]),s:o(h[7]),w:o(h[8])}):null==r?r={}:"object"==typeof r&&("from"in r||"to"in r)&&(n=M(Ce(r.from),Ce(r.to)),r={},r.ms=n.milliseconds,r.M=n.months),s=new y(r),Ce.isDuration(t)&&a(t,"_locale")&&(s._locale=t._locale),s},Ce.version=Ee,Ce.defaultFormat=ci,Ce.ISO_8601=function(){},Ce.momentProperties=Ye,Ce.updateOffset=function(){},Ce.relativeTimeThreshold=function(t,e){return bi[t]===n?!1:e===n?bi[t]:(bi[t]=e,!0)},Ce.lang=l("moment.lang is deprecated. Use moment.locale instead.",function(t,e){return Ce.locale(t,e)}),Ce.locale=function(t,e){var i;return t&&(i="undefined"!=typeof e?Ce.defineLocale(t,e):Ce.localeData(t),i&&(Ce.duration._locale=Ce._locale=i)),Ce._locale._abbr},Ce.defineLocale=function(t,e){return null!==e?(e.abbr=t,Be[t]||(Be[t]=new g),Be[t].set(e),Ce.locale(t),Be[t]):(delete Be[t],null)},Ce.langData=l("moment.langData is deprecated. Use moment.localeData instead.",function(t){return Ce.localeData(t)}),Ce.localeData=function(t){var e;if(t&&t._locale&&t._locale._abbr&&(t=t._locale._abbr),!t)return Ce._locale;if(!T(t)){if(e=W(t))return e;t=[t]}return Y(t)},Ce.isMoment=function(t){return t instanceof v||null!=t&&a(t,"_isAMomentObject")},Ce.isDuration=function(t){return t instanceof y};for(Oe=Mi.length-1;Oe>=0;--Oe)I(Mi[Oe]);Ce.normalizeUnits=function(t){return k(t)},Ce.invalid=function(t){var e=Ce.utc(0/0);return null!=t?b(e._pf,t):e._pf.userInvalidated=!0,e},Ce.parseZone=function(){return Ce.apply(null,arguments).parseZone()},Ce.parseTwoDigitYear=function(t){return L(t)+(L(t)>68?1900:2e3)},Ce.isDate=O,b(Ce.fn=v.prototype,{clone:function(){return Ce(this)},valueOf:function(){return+this._d-6e4*(this._offset||0)},unix:function(){return Math.floor(+this/1e3)},toString:function(){return this.clone().locale("en").format("ddd MMM DD YYYY HH:mm:ss [GMT]ZZ")},toDate:function(){return this._offset?new Date(+this):this._d},toISOString:function(){var t=Ce(this).utc();return 00:!1},parsingFlags:function(){return b({},this._pf)},invalidAt:function(){return this._pf.overflow},utc:function(t){return this.utcOffset(0,t)},local:function(t){return this._isUTC&&(this.utcOffset(0,t),this._isUTC=!1,t&&this.subtract(this._dateUtcOffset(),"m")),this},format:function(t){var e=V(this,t||Ce.defaultFormat);return this.localeData().postformat(e)},add:D(1,"add"),subtract:D(-1,"subtract"),diff:function(t,e,i){var s,o,n=G(t,this),r=6e4*(n.utcOffset()-this.utcOffset());return e=k(e),"year"===e||"month"===e||"quarter"===e?(o=m(this,n),"quarter"===e?o/=3:"year"===e&&(o/=12)):(s=this-n,o="second"===e?s/1e3:"minute"===e?s/6e4:"hour"===e?s/36e5:"day"===e?(s-r)/864e5:"week"===e?(s-r)/6048e5:s),i?o:x(o)},from:function(t,e){return Ce.duration({to:this,from:t}).locale(this.locale()).humanize(!e)},fromNow:function(t){return this.from(Ce(),t)},calendar:function(t){var e=t||Ce(),i=G(e,this).startOf("day"),s=this.diff(i,"days",!0),o=-6>s?"sameElse":-1>s?"lastWeek":0>s?"lastDay":1>s?"sameDay":2>s?"nextDay":7>s?"nextWeek":"sameElse";return this.format(this.localeData().calendar(o,this,Ce(e)))},isLeapYear:function(){return R(this.year())},isDST:function(){return this.utcOffset()>this.clone().month(0).utcOffset()||this.utcOffset()>this.clone().month(5).utcOffset()},day:function(t){var e=this._isUTC?this._d.getUTCDay():this._d.getDay();return null!=t?(t=ce(t,this.localeData()),this.add(t-e,"d")):e},month:xe("Month",!0),startOf:function(t){switch(t=k(t)){case"year":this.month(0);case"quarter":case"month":this.date(1);case"week":case"isoWeek":case"day":this.hours(0);case"hour":this.minutes(0);case"minute":this.seconds(0);case"second":this.milliseconds(0)}return"week"===t?this.weekday(0):"isoWeek"===t&&this.isoWeekday(1),"quarter"===t&&this.month(3*Math.floor(this.month()/3)),this},endOf:function(t){return t=k(t),t===n||"millisecond"===t?this:this.startOf(t).add(1,"isoWeek"===t?"week":t).subtract(1,"ms")},isAfter:function(t,e){var i;return e=k("undefined"!=typeof e?e:"millisecond"),"millisecond"===e?(t=Ce.isMoment(t)?t:Ce(t),+this>+t):(i=Ce.isMoment(t)?+t:+Ce(t),i<+this.clone().startOf(e))},isBefore:function(t,e){var i;return e=k("undefined"!=typeof e?e:"millisecond"),"millisecond"===e?(t=Ce.isMoment(t)?t:Ce(t),+t>+this):(i=Ce.isMoment(t)?+t:+Ce(t),+this.clone().endOf(e)t?this:t}),max:l("moment().max is deprecated, use moment.max instead. https://github.com/moment/moment/issues/1548",function(t){return t=Ce.apply(null,arguments),t>this?this:t}),zone:l("moment().zone is deprecated, use moment().utcOffset instead. https://github.com/moment/moment/issues/1779",function(t,e){return null!=t?("string"!=typeof t&&(t=-t),this.utcOffset(t,e),this):-this.utcOffset()}),utcOffset:function(t,e){var i,s=this._offset||0;return null!=t?("string"==typeof t&&(t=Z(t)),Math.abs(t)<16&&(t=60*t),!this._isUTC&&e&&(i=this._dateUtcOffset()),this._offset=t,this._isUTC=!0,null!=i&&this.add(i,"m"),s!==t&&(!e||this._changeInProgress?C(this,Ce.duration(t-s,"m"),1,!1):this._changeInProgress||(this._changeInProgress=!0,Ce.updateOffset(this,!0),this._changeInProgress=null)),this):this._isUTC?s:this._dateUtcOffset()},isLocal:function(){return!this._isUTC},isUtcOffset:function(){return this._isUTC},isUtc:function(){return this._isUTC&&0===this._offset},zoneAbbr:function(){return this._isUTC?"UTC":""},zoneName:function(){return this._isUTC?"Coordinated Universal Time":""},parseZone:function(){return this._tzm?this.utcOffset(this._tzm):"string"==typeof this._i&&this.utcOffset(Z(this._i)),this},hasAlignedHourOffset:function(t){return t=t?Ce(t).utcOffset():0,(this.utcOffset()-t)%60===0},daysInMonth:function(){return z(this.year(),this.month())},dayOfYear:function(t){var e=Ne((Ce(this).startOf("day")-Ce(this).startOf("year"))/864e5)+1;return null==t?e:this.add(t-e,"d")},quarter:function(t){return null==t?Math.ceil((this.month()+1)/3):this.month(3*(t-1)+this.month()%3)},weekYear:function(t){var e=me(this,this.localeData()._week.dow,this.localeData()._week.doy).year;return null==t?e:this.add(t-e,"y")},isoWeekYear:function(t){var e=me(this,1,4).year;return null==t?e:this.add(t-e,"y")},week:function(t){var e=this.localeData().week(this);return null==t?e:this.add(7*(t-e),"d")},isoWeek:function(t){var e=me(this,1,4).week;return null==t?e:this.add(7*(t-e),"d")},weekday:function(t){var e=(this.day()+7-this.localeData()._week.dow)%7;return null==t?e:this.add(t-e,"d")},isoWeekday:function(t){return null==t?this.day()||7:this.day(this.day()%7?t:t-7)},isoWeeksInYear:function(){return P(this.year(),1,4)},weeksInYear:function(){var t=this.localeData()._week;return P(this.year(),t.dow,t.doy)},get:function(t){return t=k(t),this[t]()},set:function(t,e){var i;if("object"==typeof t)for(i in t)this.set(i,t[i]);else t=k(t),"function"==typeof this[t]&&this[t](e);return this},locale:function(t){var e;return t===n?this._locale._abbr:(e=Ce.localeData(t),null!=e&&(this._locale=e),this)},lang:l("moment().lang() is deprecated. Instead, use moment().localeData() to get the language configuration. Use moment().locale() to change languages.",function(t){return t===n?this.localeData():this.locale(t)}),localeData:function(){return this._locale},_dateUtcOffset:function(){return 15*-Math.round(this._d.getTimezoneOffset()/15)}}),Ce.fn.millisecond=Ce.fn.milliseconds=xe("Milliseconds",!1),Ce.fn.second=Ce.fn.seconds=xe("Seconds",!1),Ce.fn.minute=Ce.fn.minutes=xe("Minutes",!1),Ce.fn.hour=Ce.fn.hours=xe("Hours",!0),Ce.fn.date=xe("Date",!0),Ce.fn.dates=l("dates accessor is deprecated. Use date instead.",xe("Date",!0)),Ce.fn.year=xe("FullYear",!0),Ce.fn.years=l("years accessor is deprecated. Use year instead.",xe("FullYear",!0)),Ce.fn.days=Ce.fn.day,Ce.fn.months=Ce.fn.month,Ce.fn.weeks=Ce.fn.week,Ce.fn.isoWeeks=Ce.fn.isoWeek,Ce.fn.quarters=Ce.fn.quarter,Ce.fn.toJSON=Ce.fn.toISOString,Ce.fn.isUTC=Ce.fn.isUtc,b(Ce.duration.fn=y.prototype,{_bubble:function(){var t,e,i,s=this._milliseconds,o=this._days,n=this._months,r=this._data,a=0;r.milliseconds=s%1e3,t=x(s/1e3),r.seconds=t%60,e=x(t/60),r.minutes=e%60,i=x(e/60),r.hours=i%24,o+=x(i/24),a=x(we(o)),o-=x(Se(a)),n+=x(o/30),o%=30,a+=x(n/12),n%=12,r.days=o,r.months=n,r.years=a},abs:function(){return this._milliseconds=Math.abs(this._milliseconds),this._days=Math.abs(this._days),this._months=Math.abs(this._months),this._data.milliseconds=Math.abs(this._data.milliseconds),this._data.seconds=Math.abs(this._data.seconds),this._data.minutes=Math.abs(this._data.minutes),this._data.hours=Math.abs(this._data.hours),this._data.months=Math.abs(this._data.months),this._data.years=Math.abs(this._data.years),this},weeks:function(){return x(this.days()/7)},valueOf:function(){return this._milliseconds+864e5*this._days+this._months%12*2592e6+31536e6*L(this._months/12)},humanize:function(t){var e=ue(this,!t,this.localeData());return t&&(e=this.localeData().pastFuture(+this,e)),this.localeData().postformat(e)},add:function(t,e){var i=Ce.duration(t,e);return this._milliseconds+=i._milliseconds,this._days+=i._days,this._months+=i._months,this._bubble(),this},subtract:function(t,e){var i=Ce.duration(t,e);return this._milliseconds-=i._milliseconds,this._days-=i._days,this._months-=i._months,this._bubble(),this},get:function(t){return t=k(t),this[t.toLowerCase()+"s"]()},as:function(t){var e,i;if(t=k(t),"month"===t||"year"===t)return e=this._days+this._milliseconds/864e5,i=this._months+12*we(e),"month"===t?i:i/12;switch(e=this._days+Math.round(Se(this._months/12)),t){case"week":return e/7+this._milliseconds/6048e5;case"day":return e+this._milliseconds/864e5;case"hour":return 24*e+this._milliseconds/36e5;case"minute":return 24*e*60+this._milliseconds/6e4;case"second":return 24*e*60*60+this._milliseconds/1e3;case"millisecond":return Math.floor(24*e*60*60*1e3)+this._milliseconds;default:throw new Error("Unknown unit "+t)}},lang:Ce.fn.lang,locale:Ce.fn.locale,toIsoString:l("toIsoString() is deprecated. Please use toISOString() instead (notice the capitals)",function(){return this.toISOString()}),toISOString:function(){var t=Math.abs(this.years()),e=Math.abs(this.months()),i=Math.abs(this.days()),s=Math.abs(this.hours()),o=Math.abs(this.minutes()),n=Math.abs(this.seconds()+this.milliseconds()/1e3);return this.asSeconds()?(this.asSeconds()<0?"-":"")+"P"+(t?t+"Y":"")+(e?e+"M":"")+(i?i+"D":"")+(s||o||n?"T":"")+(s?s+"H":"")+(o?o+"M":"")+(n?n+"S":""):"P0D"},localeData:function(){return this._locale},toJSON:function(){return this.toISOString()}}),Ce.duration.fn.toString=Ce.duration.fn.toISOString;for(Oe in fi)a(fi,Oe)&&Me(Oe.toLowerCase());Ce.duration.fn.asMilliseconds=function(){return this.as("ms")},Ce.duration.fn.asSeconds=function(){return this.as("s")},Ce.duration.fn.asMinutes=function(){return this.as("m")},Ce.duration.fn.asHours=function(){return this.as("h")},Ce.duration.fn.asDays=function(){return this.as("d")},Ce.duration.fn.asWeeks=function(){return this.as("weeks")},Ce.duration.fn.asMonths=function(){return this.as("M")},Ce.duration.fn.asYears=function(){return this.as("y")},Ce.locale("en",{ordinalParse:/\d{1,2}(th|st|nd|rd)/,ordinal:function(t){var e=t%10,i=1===L(t%100/10)?"th":1===e?"st":2===e?"nd":3===e?"rd":"th";return t+i}}),We?o.exports=Ce:(s=function(t,e,i){return i.config&&i.config()&&i.config().noGlobal===!0&&(ke.moment=Te),Ce}.call(e,i,e,o),!(s!==n&&(o.exports=s)),De(!0))}).call(this)}).call(e,function(){return this}(),i(72)(t))},function(t,e){var i,s,o;!function(n,r){s=[],i=r,o="function"==typeof i?i.apply(e,s):i,!(void 0!==o&&(t.exports=o))}(this,function(){function t(t){var e,i=t&&t.preventDefault||!1,s=t&&t.container||window,o={},n={keydown:{},keyup:{}},r={};for(e=97;122>=e;e++)r[String.fromCharCode(e)]={code:65+(e-97),shift:!1};for(e=65;90>=e;e++)r[String.fromCharCode(e)]={code:e,shift:!0};for(e=0;9>=e;e++)r[""+e]={code:48+e,shift:!1};for(e=1;12>=e;e++)r["F"+e]={code:111+e,shift:!1};for(e=0;9>=e;e++)r["num"+e]={code:96+e,shift:!1};r["num*"]={code:106,shift:!1},r["num+"]={code:107,shift:!1},r["num-"]={code:109,shift:!1},r["num/"]={code:111,shift:!1},r["num."]={code:110,shift:!1},r.left={code:37,shift:!1},r.up={code:38,shift:!1},r.right={code:39,shift:!1},r.down={code:40,shift:!1},r.space={code:32,shift:!1},r.enter={code:13,shift:!1},r.shift={code:16,shift:void 0},r.esc={code:27,shift:!1},r.backspace={code:8,shift:!1},r.tab={code:9,shift:!1},r.ctrl={code:17,shift:!1},r.alt={code:18,shift:!1},r["delete"]={code:46,shift:!1},r.pageup={code:33,shift:!1},r.pagedown={code:34,shift:!1},r["="]={code:187,shift:!1},r["-"]={code:189,shift:!1},r["]"]={code:221,shift:!1},r["["]={code:219,shift:!1};var a=function(t){d(t,"keydown")},h=function(t){d(t,"keyup")},d=function(t,e){if(void 0!==n[e][t.keyCode]){for(var s=n[e][t.keyCode],o=0;o0?i._handlers[t]=s:(i._off(t,o),delete i._handlers[t]))}),i},i.destroy=function(){var t=i.element;delete t.hammer,i._handlers={},i._destroy()},i}})},function(t,e,i){var s;!function(o,n,r,a){function h(t,e,i){return setTimeout(m(t,i),e)}function d(t,e,i){return Array.isArray(t)?(l(t,i[e],i),!0):!1}function l(t,e,i){var s;if(t)if(t.forEach)t.forEach(e,i);else if(t.length!==a)for(s=0;s-1}function x(t){return t.trim().split(/\s+/g)}function w(t,e,i){if(t.indexOf&&!i)return t.indexOf(e);for(var s=0;si[e]}):s.sort()),s}function D(t,e){for(var i,s,o=e[0].toUpperCase()+e.slice(1),n=0;n1&&!i.firstMultiple?i.firstMultiple=z(e):1===o&&(i.firstMultiple=!1);var n=i.firstInput,r=i.firstMultiple,a=r?r.center:n.center,h=e.center=P(s);e.timeStamp=ve(),e.deltaTime=e.timeStamp-n.timeStamp,e.angle=H(a,h),e.distance=F(a,h),I(i,e),e.offsetDirection=R(e.deltaX,e.deltaY),e.scale=r?Y(r.pointers,s):1,e.rotation=r?B(r.pointers,s):0,L(i,e);var d=t.element;b(e.srcEvent.target,d)&&(d=e.srcEvent.target),e.target=d}function I(t,e){var i=e.center,s=t.offsetDelta||{},o=t.prevDelta||{},n=t.prevInput||{};(e.eventType===Oe||n.eventType===ke)&&(o=t.prevDelta={x:n.deltaX||0,y:n.deltaY||0},s=t.offsetDelta={x:i.x,y:i.y}),e.deltaX=o.x+(i.x-s.x),e.deltaY=o.y+(i.y-s.y)}function L(t,e){var i,s,o,n,r=t.lastInterval||e,h=e.timeStamp-r.timeStamp;if(e.eventType!=Ne&&(h>Te||r.velocity===a)){var d=r.deltaX-e.deltaX,l=r.deltaY-e.deltaY,c=A(h,d,l);s=c.x,o=c.y,i=ge(c.x)>ge(c.y)?c.x:c.y,n=R(d,l),t.lastInterval=e}else i=r.velocity,s=r.velocityX,o=r.velocityY,n=r.direction;e.velocity=i,e.velocityX=s,e.velocityY=o,e.direction=n}function z(t){for(var e=[],i=0;io;)i+=t[o].clientX,s+=t[o].clientY,o++;return{x:fe(i/e),y:fe(s/e)}}function A(t,e,i){return{x:e/t||0,y:i/t||0}}function R(t,e){return t===e?Ie:ge(t)>=ge(e)?t>0?Le:ze:e>0?Pe:Ae}function F(t,e,i){i||(i=Be);var s=e[i[0]]-t[i[0]],o=e[i[1]]-t[i[1]];return Math.sqrt(s*s+o*o)}function H(t,e,i){i||(i=Be);var s=e[i[0]]-t[i[0]],o=e[i[1]]-t[i[1]];return 180*Math.atan2(o,s)/Math.PI}function B(t,e){return H(e[1],e[0],Ye)-H(t[1],t[0],Ye)}function Y(t,e){return F(e[0],e[1],Ye)/F(t[0],t[1],Ye)}function W(){this.evEl=Ge,this.evWin=je,this.allow=!0,this.pressed=!1,O.apply(this,arguments)}function G(){this.evEl=Xe,this.evWin=qe,O.apply(this,arguments),this.store=this.manager.session.pointerEvents=[]}function j(){this.evTarget=Qe,this.evWin=Ke,this.started=!1,O.apply(this,arguments)}function U(t,e){var i=S(t.touches),s=S(t.changedTouches);return e&(ke|Ne)&&(i=M(i.concat(s),"identifier",!0)),[i,s]}function V(){this.evTarget=Je,this.targetIds={},O.apply(this,arguments)}function X(t,e){var i=S(t.touches),s=this.targetIds;if(e&(Oe|Ee)&&1===i.length)return s[i[0].identifier]=!0,[i,i];var o,n,r=S(t.changedTouches),a=[],h=this.target;if(n=i.filter(function(t){return b(t.target,h)}),e===Oe)for(o=0;oa&&(e.push(t),a=e.length-1):o&(ke|Ne)&&(i=!0),0>a||(e[a]=t,this.callback(this.manager,o,{pointers:e,changedPointers:[t],pointerType:n,srcEvent:t}),i&&e.splice(a,1))}});var Ze={touchstart:Oe,touchmove:Ee,touchend:ke,touchcancel:Ne},Qe="touchstart",Ke="touchstart touchmove touchend touchcancel";u(j,O,{handler:function(t){var e=Ze[t.type];if(e===Oe&&(this.started=!0),this.started){var i=U.call(this,t,e);e&(ke|Ne)&&i[0].length-i[1].length===0&&(this.started=!1),this.callback(this.manager,e,{pointers:i[0],changedPointers:i[1],pointerType:Se,srcEvent:t})}}});var $e={touchstart:Oe,touchmove:Ee,touchend:ke,touchcancel:Ne},Je="touchstart touchmove touchend touchcancel";u(V,O,{handler:function(t){var e=$e[t.type],i=X.call(this,t,e);i&&this.callback(this.manager,e,{pointers:i[0],changedPointers:i[1],pointerType:Se,srcEvent:t})}}),u(q,O,{handler:function(t,e,i){var s=i.pointerType==Se,o=i.pointerType==De;if(s)this.mouse.allow=!1;else if(o&&!this.mouse.allow)return;e&(ke|Ne)&&(this.mouse.allow=!0),this.callback(t,e,i)},destroy:function(){this.touch.destroy(),this.mouse.destroy()}});var ti=D(ue.style,"touchAction"),ei=ti!==a,ii="compute",si="auto",oi="manipulation",ni="none",ri="pan-x",ai="pan-y";Z.prototype={set:function(t){t==ii&&(t=this.compute()),ei&&(this.manager.element.style[ti]=t),this.actions=t.toLowerCase().trim()},update:function(){this.set(this.manager.options.touchAction)},compute:function(){var t=[];return l(this.manager.recognizers,function(e){f(e.options.enable,[e])&&(t=t.concat(e.getTouchAction()))}),Q(t.join(" "))},preventDefaults:function(t){if(!ei){var e=t.srcEvent,i=t.offsetDirection;if(this.manager.session.prevented)return void e.preventDefault();var s=this.actions,o=_(s,ni),n=_(s,ai),r=_(s,ri);return o||n&&i&Re||r&&i&Fe?this.preventSrc(e):void 0}},preventSrc:function(t){this.manager.session.prevented=!0,t.preventDefault()}};var hi=1,di=2,li=4,ci=8,pi=ci,ui=16,mi=32;K.prototype={defaults:{},set:function(t){return c(this.options,t),this.manager&&this.manager.touchAction.update(),this},recognizeWith:function(t){if(d(t,"recognizeWith",this))return this;var e=this.simultaneous;return t=te(t,this),e[t.id]||(e[t.id]=t,t.recognizeWith(this)),this},dropRecognizeWith:function(t){return d(t,"dropRecognizeWith",this)?this:(t=te(t,this),delete this.simultaneous[t.id],this)},requireFailure:function(t){if(d(t,"requireFailure",this))return this;var e=this.requireFail;return t=te(t,this),-1===w(e,t)&&(e.push(t),t.requireFailure(this)),this},dropRequireFailure:function(t){if(d(t,"dropRequireFailure",this))return this;t=te(t,this);var e=w(this.requireFail,t);return e>-1&&this.requireFail.splice(e,1),this},hasRequireFailures:function(){return this.requireFail.length>0},canRecognizeWith:function(t){return!!this.simultaneous[t.id]},emit:function(t){function e(e){i.manager.emit(i.options.event+(e?$(s):""),t)}var i=this,s=this.state;ci>s&&e(!0),e(),s>=ci&&e(!0)},tryEmit:function(t){return this.canEmit()?this.emit(t):void(this.state=mi)},canEmit:function(){for(var t=0;tn?Le:ze,i=n!=this.pX,s=Math.abs(t.deltaX)):(o=0===r?Ie:0>r?Pe:Ae,i=r!=this.pY,s=Math.abs(t.deltaY))),t.direction=o,i&&s>e.threshold&&o&e.direction},attrTest:function(t){return ee.prototype.attrTest.call(this,t)&&(this.state&di||!(this.state&di)&&this.directionTest(t))},emit:function(t){this.pX=t.deltaX,this.pY=t.deltaY;var e=J(t.direction);e&&this.manager.emit(this.options.event+e,t),this._super.emit.call(this,t)}}),u(se,ee,{defaults:{event:"pinch",threshold:0,pointers:2},getTouchAction:function(){return[ni]},attrTest:function(t){return this._super.attrTest.call(this,t)&&(Math.abs(t.scale-1)>this.options.threshold||this.state&di)},emit:function(t){if(this._super.emit.call(this,t),1!==t.scale){var e=t.scale<1?"in":"out";this.manager.emit(this.options.event+e,t)}}}),u(oe,K,{defaults:{event:"press",pointers:1,time:500,threshold:5},getTouchAction:function(){return[si]},process:function(t){var e=this.options,i=t.pointers.length===e.pointers,s=t.distancee.time;if(this._input=t,!s||!i||t.eventType&(ke|Ne)&&!o)this.reset();else if(t.eventType&Oe)this.reset(),this._timer=h(function(){this.state=pi,this.tryEmit() -},e.time,this);else if(t.eventType&ke)return pi;return mi},reset:function(){clearTimeout(this._timer)},emit:function(t){this.state===pi&&(t&&t.eventType&ke?this.manager.emit(this.options.event+"up",t):(this._input.timeStamp=ve(),this.manager.emit(this.options.event,this._input)))}}),u(ne,ee,{defaults:{event:"rotate",threshold:0,pointers:2},getTouchAction:function(){return[ni]},attrTest:function(t){return this._super.attrTest.call(this,t)&&(Math.abs(t.rotation)>this.options.threshold||this.state&di)}}),u(re,ee,{defaults:{event:"swipe",threshold:10,velocity:.65,direction:Re|Fe,pointers:1},getTouchAction:function(){return ie.prototype.getTouchAction.call(this)},attrTest:function(t){var e,i=this.options.direction;return i&(Re|Fe)?e=t.velocity:i&Re?e=t.velocityX:i&Fe&&(e=t.velocityY),this._super.attrTest.call(this,t)&&i&t.direction&&t.distance>this.options.threshold&&ge(e)>this.options.velocity&&t.eventType&ke},emit:function(t){var e=J(t.direction);e&&this.manager.emit(this.options.event+e,t),this.manager.emit(this.options.event,t)}}),u(ae,K,{defaults:{event:"tap",pointers:1,taps:1,interval:300,time:250,threshold:2,posThreshold:10},getTouchAction:function(){return[oi]},process:function(t){var e=this.options,i=t.pointers.length===e.pointers,s=t.distancet&&s>o;)o%3==0?(this.forceAggregateHubs(!0),this.normalizeClusterLevels()):this.increaseClusterLevel(),i=this.nodeIndices.length,o+=1;o>0&&1==e&&this.repositionNodes(),this._updateCalculationNodes()},e.openCluster=function(t){var e=this.moving;if(t.clusterSize>this.constants.clustering.sectorThreshold&&this._nodeInActiveArea(t)&&("default"!=this._sector()||1!=this.nodeIndices.length)){this._addSector(t);for(var i=0;this.nodeIndices.lengthi;)this.decreaseClusterLevel(),i+=1}else this._expandClusterNode(t,!1,!0),this._updateNodeIndexList(),this._updateDynamicEdges(),this._updateCalculationNodes(),this.updateLabels();this.moving!=e&&this.start()},e.updateClustersDefault=function(){1==this.constants.clustering.enabled&&this.updateClusters(0,!1,!1)},e.increaseClusterLevel=function(){this.updateClusters(-1,!1,!0)},e.decreaseClusterLevel=function(){this.updateClusters(1,!1,!0)},e.updateClusters=function(t,e,i,s){var o=this.moving,n=this.nodeIndices.length;this.previousScale>this.scale&&0==t&&this._collapseSector(),this.previousScale>this.scale||-1==t?this._formClusters(i):(this.previousScalethis.scale||-1==t)&&(this._aggregateHubs(i),this._updateNodeIndexList()),(this.previousScale>this.scale||-1==t)&&(this.handleChains(),this._updateNodeIndexList()),this.previousScale=this.scale,this._updateDynamicEdges(),this.updateLabels(),this.nodeIndices.lengththis.constants.clustering.chainThreshold&&this._reduceAmountOfChains(1-this.constants.clustering.chainThreshold/t)},e._aggregateHubs=function(t){this._getHubSize(),this._formClustersByHub(t,!1)},e.forceAggregateHubs=function(t){var e=this.moving,i=this.nodeIndices.length;this._aggregateHubs(!0),this._updateNodeIndexList(),this._updateDynamicEdges(),this.updateLabels(),this.nodeIndices.length!=i&&(this.clusterSession+=1),(0==t||void 0===t)&&this.moving!=e&&this.start()},e._openClustersBySize=function(){for(var t in this.nodes)if(this.nodes.hasOwnProperty(t)){var e=this.nodes[t];1==e.inView()&&(e.width*this.scale>this.constants.clustering.screenSizeThreshold*this.frame.canvas.clientWidth||e.height*this.scale>this.constants.clustering.screenSizeThreshold*this.frame.canvas.clientHeight)&&this.openCluster(e)}},e._openClusters=function(t,e){for(var i=0;i1&&(t.clusterSizei)){var r=n.from,a=n.to;n.to.options.mass>n.from.options.mass&&(r=n.to,a=n.from),1==a.dynamicEdgesLength?this._addToCluster(r,a,!1):1==r.dynamicEdgesLength&&this._addToCluster(a,r,!1)}}},e._forceClustersByZoom=function(){for(var t in this.nodes)if(this.nodes.hasOwnProperty(t)){var e=this.nodes[t];if(1==e.dynamicEdgesLength&&0!=e.dynamicEdges.length){var i=e.dynamicEdges[0],s=i.toId==e.id?this.nodes[i.fromId]:this.nodes[i.toId];e.id!=s.id&&(s.options.mass>e.options.mass?this._addToCluster(s,e,!0):this._addToCluster(e,s,!0))}}},e._clusterToSmallestNeighbour=function(t){for(var e=-1,i=null,s=0;so.clusterSessions.length&&(e=o.clusterSessions.length,i=o)}null!=o&&void 0!==this.nodes[o.id]&&this._addToCluster(o,t,!0)},e._formClustersByHub=function(t,e){for(var i in this.nodes)this.nodes.hasOwnProperty(i)&&this._formClusterFromHub(this.nodes[i],t,e)},e._formClusterFromHub=function(t,e,i,s){if(void 0===s&&(s=0),t.dynamicEdgesLength>=this.hubThreshold&&0==i||t.dynamicEdgesLength==this.hubThreshold&&1==i){for(var o,n,r,a=this.constants.clustering.clusterEdgeThreshold/this.scale,h=!1,d=[],l=t.dynamicEdges.length,c=0;l>c;c++)d.push(t.dynamicEdges[c].id);if(0==e)for(h=!1,c=0;l>c;c++){var p=this.edges[d[c]];if(void 0!==p&&p.connected&&p.toId!=p.fromId&&(o=p.to.x-p.from.x,n=p.to.y-p.from.y,r=Math.sqrt(o*o+n*n),a>r)){h=!0;break}}if(!e&&h||e)for(c=0;l>c;c++)if(p=this.edges[d[c]],void 0!==p){var u=this.nodes[p.fromId==t.id?p.toId:p.fromId];u.dynamicEdges.length<=this.hubThreshold+s&&u.id!=t.id&&this._addToCluster(t,u,e)}}},e._addToCluster=function(t,e,i){t.containedNodes[e.id]=e;for(var s=0;s1)for(var s=0;s1&&(e.label="[".concat(String(e.clusterSize),"]"))}for(t in this.nodes)this.nodes.hasOwnProperty(t)&&(e=this.nodes[t],1==e.clusterSize&&(e.label=void 0!==e.originalLabel?e.originalLabel:String(e.id)))},e.normalizeClusterLevels=function(){var t,e=0,i=1e9,s=0;for(t in this.nodes)this.nodes.hasOwnProperty(t)&&(s=this.nodes[t].clusterSessions.length,s>e&&(e=s),i>s&&(i=s));if(e-i>this.constants.clustering.clusterLevelDifference){var o=this.nodeIndices.length,n=e-this.constants.clustering.clusterLevelDifference;for(t in this.nodes)this.nodes.hasOwnProperty(t)&&this.nodes[t].clusterSessions.lengths&&(s=n.dynamicEdgesLength),t+=n.dynamicEdgesLength,e+=Math.pow(n.dynamicEdgesLength,2),i+=1}t/=i,e/=i;var r=e-Math.pow(t,2),a=Math.sqrt(r);this.hubThreshold=Math.floor(t+2*a),this.hubThreshold>s&&(this.hubThreshold=s)},e._reduceAmountOfChains=function(t){this.hubThreshold=2;var e=Math.floor(this.nodeIndices.length*t);for(var i in this.nodes)this.nodes.hasOwnProperty(i)&&2==this.nodes[i].dynamicEdgesLength&&this.nodes[i].dynamicEdges.length>=2&&e>0&&(this._formClusterFromHub(this.nodes[i],!0,!0,1),e-=1)},e._getChainFraction=function(){var t=0,e=0;for(var i in this.nodes)this.nodes.hasOwnProperty(i)&&(2==this.nodes[i].dynamicEdgesLength&&this.nodes[i].dynamicEdges.length>=2&&(t+=1),e+=1);return t/e}},function(t,e,i){var s=i(1),o=i(40);e._putDataInSector=function(){this.sectors.active[this._sector()].nodes=this.nodes,this.sectors.active[this._sector()].edges=this.edges,this.sectors.active[this._sector()].nodeIndices=this.nodeIndices},e._switchToSector=function(t,e){void 0===e||"active"==e?this._switchToActiveSector(t):this._switchToFrozenSector(t)},e._switchToActiveSector=function(t){this.nodeIndices=this.sectors.active[t].nodeIndices,this.nodes=this.sectors.active[t].nodes,this.edges=this.sectors.active[t].edges},e._switchToSupportSector=function(){this.nodeIndices=this.sectors.support.nodeIndices,this.nodes=this.sectors.support.nodes,this.edges=this.sectors.support.edges},e._switchToFrozenSector=function(t){this.nodeIndices=this.sectors.frozen[t].nodeIndices,this.nodes=this.sectors.frozen[t].nodes,this.edges=this.sectors.frozen[t].edges},e._loadLatestSector=function(){this._switchToSector(this._sector())},e._sector=function(){return this.activeSector[this.activeSector.length-1]},e._previousSector=function(){if(this.activeSector.length>1)return this.activeSector[this.activeSector.length-2];throw new TypeError("there are not enough sectors in the this.activeSector array.")},e._setActiveSector=function(t){this.activeSector.push(t)},e._forgetLastSector=function(){this.activeSector.pop()},e._createNewSector=function(t){this.sectors.active[t]={nodes:{},edges:{},nodeIndices:[],formationScale:this.scale,drawingNode:void 0},this.sectors.active[t].drawingNode=new o({id:t,color:{background:"#eaefef",border:"495c5e"}},{},{},this.constants),this.sectors.active[t].drawingNode.clusterSize=2},e._deleteActiveSector=function(t){delete this.sectors.active[t]},e._deleteFrozenSector=function(t){delete this.sectors.frozen[t]},e._freezeSector=function(t){this.sectors.frozen[t]=this.sectors.active[t],this._deleteActiveSector(t)},e._activateSector=function(t){this.sectors.active[t]=this.sectors.frozen[t],this._deleteFrozenSector(t)},e._mergeThisWithFrozen=function(t){for(var e in this.nodes)this.nodes.hasOwnProperty(e)&&(this.sectors.frozen[t].nodes[e]=this.nodes[e]);for(var i in this.edges)this.edges.hasOwnProperty(i)&&(this.sectors.frozen[t].edges[i]=this.edges[i]);for(var s=0;s1?this[t](o[0],o[1]):this[t](e))}return this._loadLatestSector(),i},e._doInSupportSector=function(t,e){var i=!1;if(void 0===e)this._switchToSupportSector(),i=this[t]();else{this._switchToSupportSector();var s=Array.prototype.splice.call(arguments,1);i=s.length>1?this[t](s[0],s[1]):this[t](e)}return this._loadLatestSector(),i},e._doInAllFrozenSectors=function(t,e){if(void 0===e)for(var i in this.sectors.frozen)this.sectors.frozen.hasOwnProperty(i)&&(this._switchToFrozenSector(i),this[t]());else for(var i in this.sectors.frozen)if(this.sectors.frozen.hasOwnProperty(i)){this._switchToFrozenSector(i);var s=Array.prototype.splice.call(arguments,1);s.length>1?this[t](s[0],s[1]):this[t](e)}this._loadLatestSector()},e._doInAllSectors=function(t,e){var i=Array.prototype.splice.call(arguments,1);void 0===e?(this._doInAllActiveSectors(t),this._doInAllFrozenSectors(t)):i.length>1?(this._doInAllActiveSectors(t,i[0],i[1]),this._doInAllFrozenSectors(t,i[0],i[1])):(this._doInAllActiveSectors(t,e),this._doInAllFrozenSectors(t,e))},e._clearNodeIndexList=function(){var t=this._sector();this.sectors.active[t].nodeIndices=[],this.nodeIndices=this.sectors.active[t].nodeIndices},e._drawSectorNodes=function(t,e){var i,s=1e9,o=-1e9,n=1e9,r=-1e9;for(var a in this.sectors[e])if(this.sectors[e].hasOwnProperty(a)&&void 0!==this.sectors[e][a].drawingNode){this._switchToSector(a,e),s=1e9,o=-1e9,n=1e9,r=-1e9;for(var h in this.nodes)this.nodes.hasOwnProperty(h)&&(i=this.nodes[h],i.resize(t),n>i.x-.5*i.width&&(n=i.x-.5*i.width),ri.y-.5*i.height&&(s=i.y-.5*i.height),o0?this.nodes[i[i.length-1]]:null},e._getEdgesOverlappingWith=function(t,e){var i=this.edges;for(var s in i)i.hasOwnProperty(s)&&i[s].isOverlappingWith(t)&&e.push(s)},e._getAllEdgesOverlappingWith=function(t){var e=[];return this._doInAllActiveSectors("_getEdgesOverlappingWith",t,e),e},e._getEdgeAt=function(t){var e=this._pointerToPositionObject(t),i=this._getAllEdgesOverlappingWith(e);return i.length>0?this.edges[i[i.length-1]]:null},e._addToSelection=function(t){t instanceof s?this.selectionObj.nodes[t.id]=t:this.selectionObj.edges[t.id]=t},e._addToHover=function(t){t instanceof s?this.hoverObj.nodes[t.id]=t:this.hoverObj.edges[t.id]=t},e._removeFromSelection=function(t){t instanceof s?delete this.selectionObj.nodes[t.id]:delete this.selectionObj.edges[t.id]},e._unselectAll=function(t){void 0===t&&(t=!1);for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&this.selectionObj.nodes[e].unselect();for(var i in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(i)&&this.selectionObj.edges[i].unselect();this.selectionObj={nodes:{},edges:{}},0==t&&this.emit("select",this.getSelection())},e._unselectClusters=function(t){void 0===t&&(t=!1);for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&this.selectionObj.nodes[e].clusterSize>1&&(this.selectionObj.nodes[e].unselect(),this._removeFromSelection(this.selectionObj.nodes[e]));0==t&&this.emit("select",this.getSelection())},e._getSelectedNodeCount=function(){var t=0;for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&(t+=1);return t},e._getSelectedNode=function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t))return this.selectionObj.nodes[t];return null},e._getSelectedEdge=function(){for(var t in this.selectionObj.edges)if(this.selectionObj.edges.hasOwnProperty(t))return this.selectionObj.edges[t];return null},e._getSelectedEdgeCount=function(){var t=0;for(var e in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(e)&&(t+=1);return t},e._getSelectedObjectCount=function(){var t=0;for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&(t+=1);for(var i in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(i)&&(t+=1);return t},e._selectionIsEmpty=function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t))return!1;for(var e in this.selectionObj.edges)if(this.selectionObj.edges.hasOwnProperty(e))return!1;return!0},e._clusterInSelection=function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t)&&this.selectionObj.nodes[t].clusterSize>1)return!0;return!1},e._selectConnectedEdges=function(t){for(var e=0;ei;i++){o=t[i];var n=this.nodes[o];if(!n)throw new RangeError('Node with id "'+o+'" not found');this._selectObject(n,!0,!0,e,!0)}this.redraw()},e.selectEdges=function(t){var e,i,s;if(!t||void 0==t.length)throw"Selection must be an array with ids";for(this._unselectAll(!0),e=0,i=t.length;i>e;e++){s=t[e];var o=this.edges[s];if(!o)throw new RangeError('Edge with id "'+s+'" not found');this._selectObject(o,!0,!0,!1,!0)}this.redraw()},e._updateSelection=function(){for(var t in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(t)&&(this.nodes.hasOwnProperty(t)||delete this.selectionObj.nodes[t]);for(var e in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(e)&&(this.edges.hasOwnProperty(e)||delete this.selectionObj.edges[e])}},function(t,e,i){var s=i(1),o=i(40),n=i(37);e._clearManipulatorBar=function(){this._recursiveDOMDelete(this.manipulationDiv),this.manipulationDOM={},this._manipulationReleaseOverload=function(){},delete this.sectors.support.nodes.targetNode,delete this.sectors.support.nodes.targetViaNode,this.controlNodesActive=!1,this.freezeSimulation=!1},e._restoreOverloadedFunctions=function(){for(var t in this.cachedFunctions)this.cachedFunctions.hasOwnProperty(t)&&(this[t]=this.cachedFunctions[t],delete this.cachedFunctions[t])},e._toggleEditMode=function(){this.editMode=!this.editMode;var t=this.manipulationDiv,e=this.closeDiv,i=this.editModeDiv;1==this.editMode?(t.style.display="block",e.style.display="block",i.style.display="none",e.onclick=this._toggleEditMode.bind(this)):(t.style.display="none",e.style.display="none",i.style.display="block",e.onclick=null),this._createManipulatorBar()},e._createManipulatorBar=function(){this.boundFunction&&this.off("select",this.boundFunction);var t=this.constants.locales[this.constants.locale];if(void 0!==this.edgeBeingEdited&&(this.edgeBeingEdited._disableControlNodes(),this.edgeBeingEdited=void 0,this.selectedControlNode=null,this.controlNodesActive=!1,this._redraw()),this._restoreOverloadedFunctions(),this.freezeSimulation=!1,this.blockConnectingEdgeSelection=!1,this.forceAppendSelection=!1,this.manipulationDOM={},1==this.editMode){for(;this.manipulationDiv.hasChildNodes();)this.manipulationDiv.removeChild(this.manipulationDiv.firstChild);this.manipulationDOM.addNodeSpan=document.createElement("span"),this.manipulationDOM.addNodeSpan.className="network-manipulationUI add",this.manipulationDOM.addNodeLabelSpan=document.createElement("span"),this.manipulationDOM.addNodeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.addNodeLabelSpan.innerHTML=t.addNode,this.manipulationDOM.addNodeSpan.appendChild(this.manipulationDOM.addNodeLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.addEdgeSpan=document.createElement("span"),this.manipulationDOM.addEdgeSpan.className="network-manipulationUI connect",this.manipulationDOM.addEdgeLabelSpan=document.createElement("span"),this.manipulationDOM.addEdgeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.addEdgeLabelSpan.innerHTML=t.addEdge,this.manipulationDOM.addEdgeSpan.appendChild(this.manipulationDOM.addEdgeLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.addNodeSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.addEdgeSpan),1==this._getSelectedNodeCount()&&this.triggerFunctions.edit?(this.manipulationDOM.seperatorLineDiv2=document.createElement("div"),this.manipulationDOM.seperatorLineDiv2.className="network-seperatorLine",this.manipulationDOM.editNodeSpan=document.createElement("span"),this.manipulationDOM.editNodeSpan.className="network-manipulationUI edit",this.manipulationDOM.editNodeLabelSpan=document.createElement("span"),this.manipulationDOM.editNodeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.editNodeLabelSpan.innerHTML=t.editNode,this.manipulationDOM.editNodeSpan.appendChild(this.manipulationDOM.editNodeLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv2),this.manipulationDiv.appendChild(this.manipulationDOM.editNodeSpan)):1==this._getSelectedEdgeCount()&&0==this._getSelectedNodeCount()&&(this.manipulationDOM.seperatorLineDiv3=document.createElement("div"),this.manipulationDOM.seperatorLineDiv3.className="network-seperatorLine",this.manipulationDOM.editEdgeSpan=document.createElement("span"),this.manipulationDOM.editEdgeSpan.className="network-manipulationUI edit",this.manipulationDOM.editEdgeLabelSpan=document.createElement("span"),this.manipulationDOM.editEdgeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.editEdgeLabelSpan.innerHTML=t.editEdge,this.manipulationDOM.editEdgeSpan.appendChild(this.manipulationDOM.editEdgeLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv3),this.manipulationDiv.appendChild(this.manipulationDOM.editEdgeSpan)),0==this._selectionIsEmpty()&&(this.manipulationDOM.seperatorLineDiv4=document.createElement("div"),this.manipulationDOM.seperatorLineDiv4.className="network-seperatorLine",this.manipulationDOM.deleteSpan=document.createElement("span"),this.manipulationDOM.deleteSpan.className="network-manipulationUI delete",this.manipulationDOM.deleteLabelSpan=document.createElement("span"),this.manipulationDOM.deleteLabelSpan.className="network-manipulationLabel",this.manipulationDOM.deleteLabelSpan.innerHTML=t.del,this.manipulationDOM.deleteSpan.appendChild(this.manipulationDOM.deleteLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv4),this.manipulationDiv.appendChild(this.manipulationDOM.deleteSpan)),this.manipulationDOM.addNodeSpan.onclick=this._createAddNodeToolbar.bind(this),this.manipulationDOM.addEdgeSpan.onclick=this._createAddEdgeToolbar.bind(this),1==this._getSelectedNodeCount()&&this.triggerFunctions.edit?this.manipulationDOM.editNodeSpan.onclick=this._editNode.bind(this):1==this._getSelectedEdgeCount()&&0==this._getSelectedNodeCount()&&(this.manipulationDOM.editEdgeSpan.onclick=this._createEditEdgeToolbar.bind(this)),0==this._selectionIsEmpty()&&(this.manipulationDOM.deleteSpan.onclick=this._deleteSelected.bind(this)),this.closeDiv.onclick=this._toggleEditMode.bind(this); -var e=this;this.boundFunction=e._createManipulatorBar,this.on("select",this.boundFunction)}else{for(;this.editModeDiv.hasChildNodes();)this.editModeDiv.removeChild(this.editModeDiv.firstChild);this.manipulationDOM.editModeSpan=document.createElement("span"),this.manipulationDOM.editModeSpan.className="network-manipulationUI edit editmode",this.manipulationDOM.editModeLabelSpan=document.createElement("span"),this.manipulationDOM.editModeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.editModeLabelSpan.innerHTML=t.edit,this.manipulationDOM.editModeSpan.appendChild(this.manipulationDOM.editModeLabelSpan),this.editModeDiv.appendChild(this.manipulationDOM.editModeSpan),this.manipulationDOM.editModeSpan.onclick=this._toggleEditMode.bind(this)}},e._createAddNodeToolbar=function(){this._clearManipulatorBar(),this.boundFunction&&this.off("select",this.boundFunction);var t=this.constants.locales[this.constants.locale];this.manipulationDOM={},this.manipulationDOM.backSpan=document.createElement("span"),this.manipulationDOM.backSpan.className="network-manipulationUI back",this.manipulationDOM.backLabelSpan=document.createElement("span"),this.manipulationDOM.backLabelSpan.className="network-manipulationLabel",this.manipulationDOM.backLabelSpan.innerHTML=t.back,this.manipulationDOM.backSpan.appendChild(this.manipulationDOM.backLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.descriptionSpan=document.createElement("span"),this.manipulationDOM.descriptionSpan.className="network-manipulationUI none",this.manipulationDOM.descriptionLabelSpan=document.createElement("span"),this.manipulationDOM.descriptionLabelSpan.className="network-manipulationLabel",this.manipulationDOM.descriptionLabelSpan.innerHTML=t.addDescription,this.manipulationDOM.descriptionSpan.appendChild(this.manipulationDOM.descriptionLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.backSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.descriptionSpan),this.manipulationDOM.backSpan.onclick=this._createManipulatorBar.bind(this);var e=this;this.boundFunction=e._addNode,this.on("select",this.boundFunction)},e._createAddEdgeToolbar=function(){this._clearManipulatorBar(),this._unselectAll(!0),this.freezeSimulation=!0,this.boundFunction&&this.off("select",this.boundFunction);var t=this.constants.locales[this.constants.locale];this._unselectAll(),this.forceAppendSelection=!1,this.blockConnectingEdgeSelection=!0,this.manipulationDOM={},this.manipulationDOM.backSpan=document.createElement("span"),this.manipulationDOM.backSpan.className="network-manipulationUI back",this.manipulationDOM.backLabelSpan=document.createElement("span"),this.manipulationDOM.backLabelSpan.className="network-manipulationLabel",this.manipulationDOM.backLabelSpan.innerHTML=t.back,this.manipulationDOM.backSpan.appendChild(this.manipulationDOM.backLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.descriptionSpan=document.createElement("span"),this.manipulationDOM.descriptionSpan.className="network-manipulationUI none",this.manipulationDOM.descriptionLabelSpan=document.createElement("span"),this.manipulationDOM.descriptionLabelSpan.className="network-manipulationLabel",this.manipulationDOM.descriptionLabelSpan.innerHTML=t.edgeDescription,this.manipulationDOM.descriptionSpan.appendChild(this.manipulationDOM.descriptionLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.backSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.descriptionSpan),this.manipulationDOM.backSpan.onclick=this._createManipulatorBar.bind(this);var e=this;this.boundFunction=e._handleConnect,this.on("select",this.boundFunction),this.cachedFunctions._handleTouch=this._handleTouch,this.cachedFunctions._manipulationReleaseOverload=this._manipulationReleaseOverload,this.cachedFunctions._handleDragStart=this._handleDragStart,this.cachedFunctions._handleDragEnd=this._handleDragEnd,this._handleTouch=this._handleConnect,this._manipulationReleaseOverload=function(){},this._handleDragStart=function(){},this._handleDragEnd=this._finishConnect,this._redraw()},e._createEditEdgeToolbar=function(){this._clearManipulatorBar(),this.controlNodesActive=!0,this.boundFunction&&this.off("select",this.boundFunction),this.edgeBeingEdited=this._getSelectedEdge(),this.edgeBeingEdited._enableControlNodes();var t=this.constants.locales[this.constants.locale];this.manipulationDOM={},this.manipulationDOM.backSpan=document.createElement("span"),this.manipulationDOM.backSpan.className="network-manipulationUI back",this.manipulationDOM.backLabelSpan=document.createElement("span"),this.manipulationDOM.backLabelSpan.className="network-manipulationLabel",this.manipulationDOM.backLabelSpan.innerHTML=t.back,this.manipulationDOM.backSpan.appendChild(this.manipulationDOM.backLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.descriptionSpan=document.createElement("span"),this.manipulationDOM.descriptionSpan.className="network-manipulationUI none",this.manipulationDOM.descriptionLabelSpan=document.createElement("span"),this.manipulationDOM.descriptionLabelSpan.className="network-manipulationLabel",this.manipulationDOM.descriptionLabelSpan.innerHTML=t.editEdgeDescription,this.manipulationDOM.descriptionSpan.appendChild(this.manipulationDOM.descriptionLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.backSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.descriptionSpan),this.manipulationDOM.backSpan.onclick=this._createManipulatorBar.bind(this),this.cachedFunctions._handleTouch=this._handleTouch,this.cachedFunctions._manipulationReleaseOverload=this._manipulationReleaseOverload,this.cachedFunctions._handleTap=this._handleTap,this.cachedFunctions._handleDragStart=this._handleDragStart,this.cachedFunctions._handleOnDrag=this._handleOnDrag,this._handleTouch=this._selectControlNode,this._handleTap=function(){},this._handleOnDrag=this._controlNodeDrag,this._handleDragStart=function(){},this._manipulationReleaseOverload=this._releaseControlNode,this._redraw()},e._selectControlNode=function(t){this.edgeBeingEdited.controlNodes.from.unselect(),this.edgeBeingEdited.controlNodes.to.unselect(),this.selectedControlNode=this.edgeBeingEdited._getSelectedControlNode(this._XconvertDOMtoCanvas(t.x),this._YconvertDOMtoCanvas(t.y)),null!==this.selectedControlNode&&(this.selectedControlNode.select(),this.freezeSimulation=!0),this._redraw()},e._controlNodeDrag=function(t){var e=this._getPointer(t.center);null!==this.selectedControlNode&&void 0!==this.selectedControlNode&&(this.selectedControlNode.x=this._XconvertDOMtoCanvas(e.x),this.selectedControlNode.y=this._YconvertDOMtoCanvas(e.y)),this._redraw()},e._releaseControlNode=function(t){var e=this._getNodeAt(t);null!==e?(1==this.edgeBeingEdited.controlNodes.from.selected&&(this.edgeBeingEdited._restoreControlNodes(),this._editEdge(e.id,this.edgeBeingEdited.to.id),this.edgeBeingEdited.controlNodes.from.unselect()),1==this.edgeBeingEdited.controlNodes.to.selected&&(this.edgeBeingEdited._restoreControlNodes(),this._editEdge(this.edgeBeingEdited.from.id,e.id),this.edgeBeingEdited.controlNodes.to.unselect())):this.edgeBeingEdited._restoreControlNodes(),this.freezeSimulation=!1,this._redraw()},e._handleConnect=function(t){if(0==this._getSelectedNodeCount()){var e=this._getNodeAt(t);if(null!=e)if(e.clusterSize>1)alert(this.constants.locales[this.constants.locale].createEdgeError);else{this._selectObject(e,!1);var i=this.sectors.support.nodes;i.targetNode=new o({id:"targetNode"},{},{},this.constants);var s=i.targetNode;s.x=e.x,s.y=e.y,this.edges.connectionEdge=new n({id:"connectionEdge",from:e.id,to:s.id},this,this.constants);var r=this.edges.connectionEdge;r.from=e,r.connected=!0,r.options.smoothCurves={enabled:!0,dynamic:!1,type:"continuous",roundness:.5},r.selected=!0,r.to=s,this.cachedFunctions._handleOnDrag=this._handleOnDrag,this._handleOnDrag=function(t){var e=this._getPointer(t.center),i=this.edges.connectionEdge;i.to.x=this._XconvertDOMtoCanvas(e.x),i.to.y=this._YconvertDOMtoCanvas(e.y)},this.moving=!0,this.start()}}},e._finishConnect=function(t){if(1==this._getSelectedNodeCount()){var e=this._getPointer(t.center);this._handleOnDrag=this.cachedFunctions._handleOnDrag,delete this.cachedFunctions._handleOnDrag;var i=this.edges.connectionEdge.fromId;delete this.edges.connectionEdge,delete this.sectors.support.nodes.targetNode,delete this.sectors.support.nodes.targetViaNode;var s=this._getNodeAt(e);null!=s&&(s.clusterSize>1?alert(this.constants.locales[this.constants.locale].createEdgeError):(this._createEdge(i,s.id),this._createManipulatorBar())),this._unselectAll()}},e._addNode=function(){if(this._selectionIsEmpty()&&1==this.editMode){var t=this._pointerToPositionObject(this.pointerPosition),e={id:s.randomUUID(),x:t.left,y:t.top,label:"new",allowedToMoveX:!0,allowedToMoveY:!0};if(this.triggerFunctions.add){if(2!=this.triggerFunctions.add.length)throw new Error("The function for add does not support two arguments (data,callback)");var i=this;this.triggerFunctions.add(e,function(t){i.nodesData.add(t),i._createManipulatorBar(),i.moving=!0,i.start()})}else this.nodesData.add(e),this._createManipulatorBar(),this.moving=!0,this.start()}},e._createEdge=function(t,e){if(1==this.editMode){var i={from:t,to:e};if(this.triggerFunctions.connect){if(2!=this.triggerFunctions.connect.length)throw new Error("The function for connect does not support two arguments (data,callback)");var s=this;this.triggerFunctions.connect(i,function(t){s.edgesData.add(t),s.moving=!0,s.start()})}else this.edgesData.add(i),this.moving=!0,this.start()}},e._editEdge=function(t,e){if(1==this.editMode){var i={id:this.edgeBeingEdited.id,from:t,to:e};if(this.triggerFunctions.editEdge){if(2!=this.triggerFunctions.editEdge.length)throw new Error("The function for edit does not support two arguments (data, callback)");var s=this;this.triggerFunctions.editEdge(i,function(t){s.edgesData.update(t),s.moving=!0,s.start()})}else this.edgesData.update(i),this.moving=!0,this.start()}},e._editNode=function(){if(!this.triggerFunctions.edit||1!=this.editMode)throw new Error("No edit function has been bound to this button");var t=this._getSelectedNode(),e={id:t.id,label:t.label,group:t.options.group,shape:t.options.shape,color:{background:t.options.color.background,border:t.options.color.border,highlight:{background:t.options.color.highlight.background,border:t.options.color.highlight.border}}};if(2!=this.triggerFunctions.edit.length)throw new Error("The function for edit does not support two arguments (data, callback)");var i=this;this.triggerFunctions.edit(e,function(t){i.nodesData.update(t),i._createManipulatorBar(),i.moving=!0,i.start()})},e._deleteSelected=function(){if(!this._selectionIsEmpty()&&1==this.editMode)if(this._clusterInSelection())alert(this.constants.locales[this.constants.locale].deleteClusterError);else{var t=this.getSelectedNodes(),e=this.getSelectedEdges();if(this.triggerFunctions.del){var i=this,s={nodes:t,edges:e};if(2!=this.triggerFunctions.del.length)throw new Error("The function for delete does not support two arguments (data, callback)");this.triggerFunctions.del(s,function(t){i.edgesData.remove(t.edges),i.nodesData.remove(t.nodes),i._unselectAll(),i.moving=!0,i.start()})}else this.edgesData.remove(e),this.nodesData.remove(t),this._unselectAll(),this.moving=!0,this.start()}}},function(t,e,i){var s=(i(1),i(47)),o=i(45);e._cleanNavigation=function(){if(0!=this.navigationHammers.existing.length){for(var t=0;t0){var t,e,i=0,s=!1,o=!1;for(e in this.nodes)this.nodes.hasOwnProperty(e)&&(t=this.nodes[e],-1!=t.level?s=!0:o=!0,is&&(n.xFixed=!1,n.x=i[n.level].minPos,r=!0):n.yFixed&&n.level>s&&(n.yFixed=!1,n.y=i[n.level].minPos,r=!0),1==r&&(i[n.level].minPos+=i[n.level].nodeSpacing,n.edges.length>1&&this._placeBranchNodes(n.edges,n.id,i,n.level))}},e._setLevel=function(t,e,i){for(var s=0;st)&&(o.level=t,o.edges.length>1&&this._setLevel(t+1,o.edges,o.id))}},e._setLevelDirected=function(t,e,i){this.nodes[i].hierarchyEnumerated=!0;for(var s,o,n=0;n1&&s.hierarchyEnumerated===!1&&this._setLevelDirected(s.level,s.edges,s.id)},e._restoreNodes=function(){for(var t in this.nodes)this.nodes.hasOwnProperty(t)&&(this.nodes[t].xFixed=!1,this.nodes[t].yFixed=!1)}},function(t,e,i){function s(){this.constants.smoothCurves.enabled=!this.constants.smoothCurves.enabled;var t=document.getElementById("graph_toggleSmooth");t.style.background=1==this.constants.smoothCurves.enabled?"#A4FF56":"#FF8532",this._configureSmoothCurves(!1)}function o(){for(var t in this.calculationNodes)this.calculationNodes.hasOwnProperty(t)&&(this.calculationNodes[t].vx=0,this.calculationNodes[t].vy=0,this.calculationNodes[t].fx=0,this.calculationNodes[t].fy=0);1==this.constants.hierarchicalLayout.enabled?(this._setupHierarchicalLayout(),a.call(this,"graph_H_nd",1,"physics_hierarchicalRepulsion_nodeDistance"),a.call(this,"graph_H_cg",1,"physics_centralGravity"),a.call(this,"graph_H_sc",1,"physics_springConstant"),a.call(this,"graph_H_sl",1,"physics_springLength"),a.call(this,"graph_H_damp",1,"physics_damping")):this.repositionNodes(),this.moving=!0,this.start()}function n(){var t="No options are required, default values used.",e=[],i=document.getElementById("graph_physicsMethod1"),s=document.getElementById("graph_physicsMethod2");if(1==i.checked){if(this.constants.physics.barnesHut.gravitationalConstant!=this.backupConstants.physics.barnesHut.gravitationalConstant&&e.push("gravitationalConstant: "+this.constants.physics.barnesHut.gravitationalConstant),this.constants.physics.centralGravity!=this.backupConstants.physics.barnesHut.centralGravity&&e.push("centralGravity: "+this.constants.physics.centralGravity),this.constants.physics.springLength!=this.backupConstants.physics.barnesHut.springLength&&e.push("springLength: "+this.constants.physics.springLength),this.constants.physics.springConstant!=this.backupConstants.physics.barnesHut.springConstant&&e.push("springConstant: "+this.constants.physics.springConstant),this.constants.physics.damping!=this.backupConstants.physics.barnesHut.damping&&e.push("damping: "+this.constants.physics.damping),0!=e.length){t="var options = {",t+="physics: {barnesHut: {";for(var o=0;othis.constants.clustering.clusterThreshold&&1==this.constants.clustering.enabled&&this.clusterToFit(this.constants.clustering.reduceToNodes,!1),this._calculateForces())},e._calculateForces=function(){this._calculateGravitationalForces(),this._calculateNodeForces(),this.constants.physics.springConstant>0&&(1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic?this._calculateSpringForcesWithSupport():1==this.constants.physics.hierarchicalRepulsion.enabled?this._calculateHierarchicalSpringForces():this._calculateSpringForces())},e._updateCalculationNodes=function(){if(1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic){this.calculationNodes={},this.calculationNodeIndices=[];for(var t in this.nodes)this.nodes.hasOwnProperty(t)&&(this.calculationNodes[t]=this.nodes[t]);var e=this.sectors.support.nodes;for(var i in e)e.hasOwnProperty(i)&&(this.edges.hasOwnProperty(e[i].parentEdgeId)?this.calculationNodes[i]=e[i]:e[i]._setForce(0,0));for(var s in this.calculationNodes)this.calculationNodes.hasOwnProperty(s)&&this.calculationNodeIndices.push(s)}else this.calculationNodes=this.nodes,this.calculationNodeIndices=this.nodeIndices},e._calculateGravitationalForces=function(){var t,e,i,s,o,n=this.calculationNodes,r=this.constants.physics.centralGravity,a=0;for(o=0;oSimulation Mode:Barnes HutRepulsionHierarchical
Options:
',this.containerElement.parentElement.insertBefore(this.physicsConfiguration,this.containerElement),this.optionsDiv=document.createElement("div"),this.optionsDiv.style.fontSize="14px",this.optionsDiv.style.fontFamily="verdana",this.containerElement.parentElement.insertBefore(this.optionsDiv,this.containerElement); -var e;e=document.getElementById("graph_BH_gc"),e.onchange=a.bind(this,"graph_BH_gc",-1,"physics_barnesHut_gravitationalConstant"),e=document.getElementById("graph_BH_cg"),e.onchange=a.bind(this,"graph_BH_cg",1,"physics_centralGravity"),e=document.getElementById("graph_BH_sc"),e.onchange=a.bind(this,"graph_BH_sc",1,"physics_springConstant"),e=document.getElementById("graph_BH_sl"),e.onchange=a.bind(this,"graph_BH_sl",1,"physics_springLength"),e=document.getElementById("graph_BH_damp"),e.onchange=a.bind(this,"graph_BH_damp",1,"physics_damping"),e=document.getElementById("graph_R_nd"),e.onchange=a.bind(this,"graph_R_nd",1,"physics_repulsion_nodeDistance"),e=document.getElementById("graph_R_cg"),e.onchange=a.bind(this,"graph_R_cg",1,"physics_centralGravity"),e=document.getElementById("graph_R_sc"),e.onchange=a.bind(this,"graph_R_sc",1,"physics_springConstant"),e=document.getElementById("graph_R_sl"),e.onchange=a.bind(this,"graph_R_sl",1,"physics_springLength"),e=document.getElementById("graph_R_damp"),e.onchange=a.bind(this,"graph_R_damp",1,"physics_damping"),e=document.getElementById("graph_H_nd"),e.onchange=a.bind(this,"graph_H_nd",1,"physics_hierarchicalRepulsion_nodeDistance"),e=document.getElementById("graph_H_cg"),e.onchange=a.bind(this,"graph_H_cg",1,"physics_centralGravity"),e=document.getElementById("graph_H_sc"),e.onchange=a.bind(this,"graph_H_sc",1,"physics_springConstant"),e=document.getElementById("graph_H_sl"),e.onchange=a.bind(this,"graph_H_sl",1,"physics_springLength"),e=document.getElementById("graph_H_damp"),e.onchange=a.bind(this,"graph_H_damp",1,"physics_damping"),e=document.getElementById("graph_H_direction"),e.onchange=a.bind(this,"graph_H_direction",t,"hierarchicalLayout_direction"),e=document.getElementById("graph_H_levsep"),e.onchange=a.bind(this,"graph_H_levsep",1,"hierarchicalLayout_levelSeparation"),e=document.getElementById("graph_H_nspac"),e.onchange=a.bind(this,"graph_H_nspac",1,"hierarchicalLayout_nodeSpacing");var i=document.getElementById("graph_physicsMethod1"),d=document.getElementById("graph_physicsMethod2"),l=document.getElementById("graph_physicsMethod3");d.checked=!0,this.constants.physics.barnesHut.enabled&&(i.checked=!0),this.constants.hierarchicalLayout.enabled&&(l.checked=!0);var c=document.getElementById("graph_toggleSmooth"),p=document.getElementById("graph_repositionNodes"),u=document.getElementById("graph_generateOptions");c.onclick=s.bind(this),p.onclick=o.bind(this),u.onclick=n.bind(this),c.style.background=1==this.constants.smoothCurves&&0==this.constants.dynamicSmoothCurves?"#A4FF56":"#FF8532",r.apply(this),i.onchange=r.bind(this),d.onchange=r.bind(this),l.onchange=r.bind(this)}},e._overWriteGraphConstants=function(t,e){var i=t.split("_");1==i.length?this.constants[i[0]]=e:2==i.length?this.constants[i[0]][i[1]]=e:3==i.length&&(this.constants[i[0]][i[1]][i[2]]=e)}},function(t){function e(t){throw new Error("Cannot find module '"+t+"'.")}e.keys=function(){return[]},e.resolve=e,t.exports=e,e.id=68},function(t,e){e._calculateNodeForces=function(){var t,e,i,s,o,n,r,a,h,d,l,c=this.calculationNodes,p=this.calculationNodeIndices,u=-2/3,m=4/3,f=this.constants.physics.repulsion.nodeDistance,g=f;for(d=0;di&&(r=.5*g>i?1:v*i+m,r*=0==n?1:1+n*this.constants.clustering.forceAmplification,r/=Math.max(i,.01*g),s=t*r,o=e*r,a.fx-=s,a.fy-=o,h.fx+=s,h.fy+=o)}}},function(t,e){e._calculateNodeForces=function(){var t,e,i,s,o,n,r,a,h,d,l=this.calculationNodes,c=this.calculationNodeIndices,p=this.constants.physics.hierarchicalRepulsion.nodeDistance;for(h=0;hi?-Math.pow(u*i,2)+Math.pow(u*p,2):0,0==i?i=.01:n/=i,s=t*n,o=e*n,r.fx-=s,r.fy-=o,a.fx+=s,a.fy+=o}},e._calculateHierarchicalSpringForces=function(){for(var t,e,i,s,o,n,r,a,h,d=this.edges,l=this.calculationNodes,c=this.calculationNodeIndices,p=0;pn;n++)t=e[i[n]],t.options.mass>0&&(this._getForceContribution(o.root.children.NW,t),this._getForceContribution(o.root.children.NE,t),this._getForceContribution(o.root.children.SW,t),this._getForceContribution(o.root.children.SE,t))}},e._getForceContribution=function(t,e){if(t.childrenCount>0){var i,s,o;if(i=t.centerOfMass.x-e.x,s=t.centerOfMass.y-e.y,o=Math.sqrt(i*i+s*s),o*t.calcSize>this.constants.physics.barnesHut.thetaInverted){0==o&&(o=.1*Math.random(),i=o);var n=this.constants.physics.barnesHut.gravitationalConstant*t.mass*e.options.mass/(o*o*o),r=i*n,a=s*n;e.fx+=r,e.fy+=a}else if(4==t.childrenCount)this._getForceContribution(t.children.NW,e),this._getForceContribution(t.children.NE,e),this._getForceContribution(t.children.SW,e),this._getForceContribution(t.children.SE,e);else if(t.children.data.id!=e.id){0==o&&(o=.5*Math.random(),i=o);var n=this.constants.physics.barnesHut.gravitationalConstant*t.mass*e.options.mass/(o*o*o),r=i*n,a=s*n;e.fx+=r,e.fy+=a}}},e._formBarnesHutTree=function(t,e){for(var i,s=e.length,o=Number.MAX_VALUE,n=Number.MAX_VALUE,r=-Number.MAX_VALUE,a=-Number.MAX_VALUE,h=0;s>h;h++){var d=t[e[h]].x,l=t[e[h]].y;t[e[h]].options.mass>0&&(o>d&&(o=d),d>r&&(r=d),n>l&&(n=l),l>a&&(a=l))}var c=Math.abs(r-o)-Math.abs(a-n);c>0?(n-=.5*c,a+=.5*c):(o+=.5*c,r-=.5*c);var p=1e-5,u=Math.max(p,Math.abs(r-o)),m=.5*u,f=.5*(o+r),g=.5*(n+a),v={root:{centerOfMass:{x:0,y:0},mass:0,range:{minX:f-m,maxX:f+m,minY:g-m,maxY:g+m},size:u,calcSize:1/u,children:{data:null},maxWidth:0,level:0,childrenCount:4}};for(this._splitBranch(v.root),h=0;s>h;h++)i=t[e[h]],i.options.mass>0&&this._placeInTree(v.root,i);this.barnesHutTree=v},e._updateBranchMass=function(t,e){var i=t.mass+e.options.mass,s=1/i;t.centerOfMass.x=t.centerOfMass.x*t.mass+e.x*e.options.mass,t.centerOfMass.x*=s,t.centerOfMass.y=t.centerOfMass.y*t.mass+e.y*e.options.mass,t.centerOfMass.y*=s,t.mass=i;var o=Math.max(Math.max(e.height,e.radius),e.width);t.maxWidth=t.maxWidthe.x?t.children.NW.range.maxY>e.y?this._placeInRegion(t,e,"NW"):this._placeInRegion(t,e,"SW"):t.children.NW.range.maxY>e.y?this._placeInRegion(t,e,"NE"):this._placeInRegion(t,e,"SE")},e._placeInRegion=function(t,e,i){switch(t.children[i].childrenCount){case 0:t.children[i].children.data=e,t.children[i].childrenCount=1,this._updateBranchMass(t.children[i],e);break;case 1:t.children[i].children.data.x==e.x&&t.children[i].children.data.y==e.y?(e.x+=Math.random(),e.y+=Math.random()):(this._splitBranch(t.children[i]),this._placeInTree(t.children[i],e));break;case 4:this._placeInTree(t.children[i],e)}},e._splitBranch=function(t){var e=null;1==t.childrenCount&&(e=t.children.data,t.mass=0,t.centerOfMass.x=0,t.centerOfMass.y=0),t.childrenCount=4,t.children.data=null,this._insertRegion(t,"NW"),this._insertRegion(t,"NE"),this._insertRegion(t,"SW"),this._insertRegion(t,"SE"),null!=e&&this._placeInTree(t,e)},e._insertRegion=function(t,e){var i,s,o,n,r=.5*t.size;switch(e){case"NW":i=t.range.minX,s=t.range.minX+r,o=t.range.minY,n=t.range.minY+r;break;case"NE":i=t.range.minX+r,s=t.range.maxX,o=t.range.minY,n=t.range.minY+r;break;case"SW":i=t.range.minX,s=t.range.minX+r,o=t.range.minY+r,n=t.range.maxY;break;case"SE":i=t.range.minX+r,s=t.range.maxX,o=t.range.minY+r,n=t.range.maxY}t.children[e]={centerOfMass:{x:0,y:0},mass:0,range:{minX:i,maxX:s,minY:o,maxY:n},size:.5*t.size,calcSize:2*t.calcSize,children:{data:null},maxWidth:0,level:t.level+1,childrenCount:0}},e._drawTree=function(t,e){void 0!==this.barnesHutTree&&(t.lineWidth=1,this._drawBranch(this.barnesHutTree.root,t,e))},e._drawBranch=function(t,e,i){void 0===i&&(i="#FF0000"),4==t.childrenCount&&(this._drawBranch(t.children.NW,e),this._drawBranch(t.children.NE,e),this._drawBranch(t.children.SE,e),this._drawBranch(t.children.SW,e)),e.strokeStyle=i,e.beginPath(),e.moveTo(t.range.minX,t.range.minY),e.lineTo(t.range.maxX,t.range.minY),e.stroke(),e.beginPath(),e.moveTo(t.range.maxX,t.range.minY),e.lineTo(t.range.maxX,t.range.maxY),e.stroke(),e.beginPath(),e.moveTo(t.range.maxX,t.range.maxY),e.lineTo(t.range.minX,t.range.maxY),e.stroke(),e.beginPath(),e.moveTo(t.range.minX,t.range.maxY),e.lineTo(t.range.minX,t.range.minY),e.stroke()}},function(t){t.exports=function(t){return t.webpackPolyfill||(t.deprecate=function(){},t.paths=[],t.children=[],t.webpackPolyfill=1),t}},function(t,e){(function(e){t.exports=e}).call(e,{})}])}); +"use strict";!function(t,e){"object"==typeof exports&&"object"==typeof module?module.exports=e():"function"==typeof define&&define.amd?define([],e):"object"==typeof exports?exports.vis=e():t.vis=e()}(this,function(){return function(t){function e(o){if(i[o])return i[o].exports;var n=i[o]={exports:{},id:o,loaded:!1};return t[o].call(n.exports,n,n.exports,e),n.loaded=!0,n.exports}var i={};return e.m=t,e.c=i,e.p="",e(0)}([function(t,e,i){var o=i(1);o.extend(e,i(7)),o.extend(e,i(24)),o.extend(e,i(60))},function(t,e,i){var o="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},n=i(2),s=i(6);e.isNumber=function(t){return t instanceof Number||"number"==typeof t},e.recursiveDOMDelete=function(t){if(t)for(;t.hasChildNodes()===!0;)e.recursiveDOMDelete(t.firstChild),t.removeChild(t.firstChild)},e.giveRange=function(t,e,i,o){if(e==t)return.5;var n=1/(e-t);return Math.max(0,(o-t)*n)},e.isString=function(t){return t instanceof String||"string"==typeof t},e.isDate=function(t){if(t instanceof Date)return!0;if(e.isString(t)){var i=r.exec(t);if(i)return!0;if(!isNaN(Date.parse(t)))return!0}return!1},e.randomUUID=function(){return s.v4()},e.assignAllKeys=function(t,e){for(var i in t)t.hasOwnProperty(i)&&"object"!==o(t[i])&&(t[i]=e)},e.fillIfDefined=function(t,i){var n=arguments.length<=2||void 0===arguments[2]?!1:arguments[2];for(var s in t)void 0!==i[s]&&("object"!==o(i[s])?void 0!==i[s]&&null!==i[s]||void 0===t[s]||n!==!0?t[s]=i[s]:delete t[s]:"object"===o(t[s])&&e.fillIfDefined(t[s],i[s],n))},e.protoExtend=function(t,e){for(var i=1;ii;i++)if(t[i]!=e[i])return!1;return!0},e.convert=function(t,i){var o;if(void 0!==t){if(null===t)return null;if(!i)return t;if("string"!=typeof i&&!(i instanceof String))throw new Error("Type must be a string");switch(i){case"boolean":case"Boolean":return Boolean(t);case"number":case"Number":return Number(t.valueOf());case"string":case"String":return String(t);case"Date":if(e.isNumber(t))return new Date(t);if(t instanceof Date)return new Date(t.valueOf());if(n.isMoment(t))return new Date(t.valueOf());if(e.isString(t))return o=r.exec(t),o?new Date(Number(o[1])):n(t).toDate();throw new Error("Cannot convert object of type "+e.getType(t)+" to type Date");case"Moment":if(e.isNumber(t))return n(t);if(t instanceof Date)return n(t.valueOf());if(n.isMoment(t))return n(t);if(e.isString(t))return o=r.exec(t),n(o?Number(o[1]):t);throw new Error("Cannot convert object of type "+e.getType(t)+" to type Date");case"ISODate":if(e.isNumber(t))return new Date(t);if(t instanceof Date)return t.toISOString();if(n.isMoment(t))return t.toDate().toISOString();if(e.isString(t))return o=r.exec(t),o?new Date(Number(o[1])).toISOString():new Date(t).toISOString();throw new Error("Cannot convert object of type "+e.getType(t)+" to type ISODate");case"ASPDate":if(e.isNumber(t))return"/Date("+t+")/";if(t instanceof Date)return"/Date("+t.valueOf()+")/";if(e.isString(t)){o=r.exec(t);var s;return s=o?new Date(Number(o[1])).valueOf():new Date(t).valueOf(),"/Date("+s+")/"}throw new Error("Cannot convert object of type "+e.getType(t)+" to type ASPDate");default:throw new Error('Unknown type "'+i+'"')}}};var r=/^\/?Date\((\-?\d+)/i;e.getType=function(t){var e="undefined"==typeof t?"undefined":o(t);return"object"==e?null===t?"null":t instanceof Boolean?"Boolean":t instanceof Number?"Number":t instanceof String?"String":Array.isArray(t)?"Array":t instanceof Date?"Date":"Object":"number"==e?"Number":"boolean"==e?"Boolean":"string"==e?"String":void 0===e?"undefined":e},e.copyAndExtendArray=function(t,e){for(var i=[],o=0;oi;i++)e(t[i],i,t);else for(i in t)t.hasOwnProperty(i)&&e(t[i],i,t)},e.toArray=function(t){var e=[];for(var i in t)t.hasOwnProperty(i)&&e.push(t[i]);return e},e.updateProperty=function(t,e,i){return t[e]!==i?(t[e]=i,!0):!1},e.throttle=function(t,e){var i=null,o=!1;return function n(){i?o=!0:(o=!1,t(),i=setTimeout(function(){i=null,o&&n()},e))}},e.addEventListener=function(t,e,i,o){t.addEventListener?(void 0===o&&(o=!1),"mousewheel"===e&&navigator.userAgent.indexOf("Firefox")>=0&&(e="DOMMouseScroll"),t.addEventListener(e,i,o)):t.attachEvent("on"+e,i)},e.removeEventListener=function(t,e,i,o){t.removeEventListener?(void 0===o&&(o=!1),"mousewheel"===e&&navigator.userAgent.indexOf("Firefox")>=0&&(e="DOMMouseScroll"),t.removeEventListener(e,i,o)):t.detachEvent("on"+e,i)},e.preventDefault=function(t){t||(t=window.event),t.preventDefault?t.preventDefault():t.returnValue=!1},e.getTarget=function(t){t||(t=window.event);var e;return t.target?e=t.target:t.srcElement&&(e=t.srcElement),void 0!=e.nodeType&&3==e.nodeType&&(e=e.parentNode),e},e.hasParent=function(t,e){for(var i=t;i;){if(i===e)return!0;i=i.parentNode}return!1},e.option={},e.option.asBoolean=function(t,e){return"function"==typeof t&&(t=t()),null!=t?0!=t:e||null},e.option.asNumber=function(t,e){return"function"==typeof t&&(t=t()),null!=t?Number(t)||e||null:e||null},e.option.asString=function(t,e){return"function"==typeof t&&(t=t()),null!=t?String(t):e||null},e.option.asSize=function(t,i){return"function"==typeof t&&(t=t()),e.isString(t)?t:e.isNumber(t)?t+"px":i||null},e.option.asElement=function(t,e){return"function"==typeof t&&(t=t()),t||e||null},e.hexToRGB=function(t){var e=/^#?([a-f\d])([a-f\d])([a-f\d])$/i;t=t.replace(e,function(t,e,i,o){return e+e+i+i+o+o});var i=/^#?([a-f\d]{2})([a-f\d]{2})([a-f\d]{2})$/i.exec(t);return i?{r:parseInt(i[1],16),g:parseInt(i[2],16),b:parseInt(i[3],16)}:null},e.overrideOpacity=function(t,i){if(-1!=t.indexOf("rgba"))return t;if(-1!=t.indexOf("rgb")){var o=t.substr(t.indexOf("(")+1).replace(")","").split(",");return"rgba("+o[0]+","+o[1]+","+o[2]+","+i+")"}var o=e.hexToRGB(t);return null==o?t:"rgba("+o.r+","+o.g+","+o.b+","+i+")"},e.RGBToHex=function(t,e,i){return"#"+((1<<24)+(t<<16)+(e<<8)+i).toString(16).slice(1)},e.parseColor=function(t){var i;if(e.isString(t)===!0){if(e.isValidRGB(t)===!0){var o=t.substr(4).substr(0,t.length-5).split(",").map(function(t){return parseInt(t)});t=e.RGBToHex(o[0],o[1],o[2])}if(e.isValidHex(t)===!0){var n=e.hexToHSV(t),s={h:n.h,s:.8*n.s,v:Math.min(1,1.02*n.v)},r={h:n.h,s:Math.min(1,1.25*n.s),v:.8*n.v},a=e.HSVToHex(r.h,r.s,r.v),h=e.HSVToHex(s.h,s.s,s.v);i={background:t,border:a,highlight:{background:h,border:a},hover:{background:h,border:a}}}else i={background:t,border:t,highlight:{background:t,border:t},hover:{background:t,border:t}}}else i={},i.background=t.background||void 0,i.border=t.border||void 0,e.isString(t.highlight)?i.highlight={border:t.highlight,background:t.highlight}:(i.highlight={},i.highlight.background=t.highlight&&t.highlight.background||void 0,i.highlight.border=t.highlight&&t.highlight.border||void 0),e.isString(t.hover)?i.hover={border:t.hover,background:t.hover}:(i.hover={},i.hover.background=t.hover&&t.hover.background||void 0,i.hover.border=t.hover&&t.hover.border||void 0);return i},e.RGBToHSV=function(t,e,i){t/=255,e/=255,i/=255;var o=Math.min(t,Math.min(e,i)),n=Math.max(t,Math.max(e,i));if(o==n)return{h:0,s:0,v:o};var s=t==o?e-i:i==o?t-e:i-t,r=t==o?3:i==o?1:5,a=60*(r-s/(n-o))/360,h=(n-o)/n,d=n;return{h:a,s:h,v:d}};var a={split:function(t){var e={};return t.split(";").forEach(function(t){if(""!=t.trim()){var i=t.split(":"),o=i[0].trim(),n=i[1].trim();e[o]=n}}),e},join:function(t){return Object.keys(t).map(function(e){return e+": "+t[e]}).join("; ")}};e.addCssText=function(t,i){var o=a.split(t.style.cssText),n=a.split(i),s=e.extend(o,n);t.style.cssText=a.join(s)},e.removeCssText=function(t,e){var i=a.split(t.style.cssText),o=a.split(e);for(var n in o)o.hasOwnProperty(n)&&delete i[n];t.style.cssText=a.join(i)},e.HSVToRGB=function(t,e,i){var o,n,s,r=Math.floor(6*t),a=6*t-r,h=i*(1-e),d=i*(1-a*e),l=i*(1-(1-a)*e);switch(r%6){case 0:o=i,n=l,s=h;break;case 1:o=d,n=i,s=h;break;case 2:o=h,n=i,s=l;break;case 3:o=h,n=d,s=i;break;case 4:o=l,n=h,s=i;break;case 5:o=i,n=h,s=d}return{r:Math.floor(255*o),g:Math.floor(255*n),b:Math.floor(255*s)}},e.HSVToHex=function(t,i,o){var n=e.HSVToRGB(t,i,o);return e.RGBToHex(n.r,n.g,n.b)},e.hexToHSV=function(t){var i=e.hexToRGB(t);return e.RGBToHSV(i.r,i.g,i.b)},e.isValidHex=function(t){var e=/(^#[0-9A-F]{6}$)|(^#[0-9A-F]{3}$)/i.test(t);return e},e.isValidRGB=function(t){t=t.replace(" ","");var e=/rgb\((\d{1,3}),(\d{1,3}),(\d{1,3})\)/i.test(t);return e},e.isValidRGBA=function(t){t=t.replace(" ","");var e=/rgba\((\d{1,3}),(\d{1,3}),(\d{1,3}),(.{1,3})\)/i.test(t);return e},e.selectiveBridgeObject=function(t,i){if("object"==("undefined"==typeof i?"undefined":o(i))){for(var n=Object.create(i),s=0;s0&&e(o,t[n-1])<0;n--)t[n]=t[n-1];t[n]=o}return t},e.mergeOptions=function(t,e,i){var o=(arguments.length<=3||void 0===arguments[3]?!1:arguments[3],arguments.length<=4||void 0===arguments[4]?{}:arguments[4]);if(null===e[i])t[i]=Object.create(o[i]);else if(void 0!==e[i])if("boolean"==typeof e[i])t[i].enabled=e[i];else{void 0===e[i].enabled&&(t[i].enabled=!0);for(var n in e[i])e[i].hasOwnProperty(n)&&(t[i][n]=e[i][n])}},e.binarySearchCustom=function(t,e,i,o){for(var n=1e4,s=0,r=0,a=t.length-1;a>=r&&n>s;){var h=Math.floor((r+a)/2),d=t[h],l=void 0===o?d[i]:d[i][o],c=e(l);if(0==c)return h;-1==c?r=h+1:a=h-1,s++}return-1},e.binarySearchValue=function(t,e,i,o,n){for(var s,r,a,h,d=1e4,l=0,c=0,u=t.length-1,n=void 0!=n?n:function(t,e){return t==e?0:e>t?-1:1};u>=c&&d>l;){if(h=Math.floor(.5*(u+c)),s=t[Math.max(0,h-1)][i],r=t[h][i],a=t[Math.min(t.length-1,h+1)][i],0==n(r,e))return h;if(n(s,e)<0&&n(r,e)>0)return"before"==o?Math.max(0,h-1):h;if(n(r,e)<0&&n(a,e)>0)return"before"==o?h:Math.min(t.length-1,h+1);n(r,e)<0?c=h+1:u=h-1,l++}return-1},e.easingFunctions={linear:function(t){return t},easeInQuad:function(t){return t*t},easeOutQuad:function(t){return t*(2-t)},easeInOutQuad:function(t){return.5>t?2*t*t:-1+(4-2*t)*t},easeInCubic:function(t){return t*t*t},easeOutCubic:function(t){return--t*t*t+1},easeInOutCubic:function(t){return.5>t?4*t*t*t:(t-1)*(2*t-2)*(2*t-2)+1},easeInQuart:function(t){return t*t*t*t},easeOutQuart:function(t){return 1- --t*t*t*t},easeInOutQuart:function(t){return.5>t?8*t*t*t*t:1-8*--t*t*t*t},easeInQuint:function(t){return t*t*t*t*t},easeOutQuint:function(t){return 1+--t*t*t*t*t},easeInOutQuint:function(t){return.5>t?16*t*t*t*t*t:1+16*--t*t*t*t*t}}},function(t,e,i){t.exports="undefined"!=typeof window&&window.moment||i(3)},function(t,e,i){(function(t){!function(e,i){t.exports=i()}(this,function(){function e(){return ro.apply(null,arguments)}function i(t){ro=t}function o(t){return t instanceof Array||"[object Array]"===Object.prototype.toString.call(t)}function n(t){return t instanceof Date||"[object Date]"===Object.prototype.toString.call(t)}function s(t,e){var i,o=[];for(i=0;i0)for(i in ho)o=ho[i],n=e[o],p(n)||(t[o]=n);return t}function m(t){f(this,t),this._d=new Date(null!=t._d?t._d.getTime():NaN),lo===!1&&(lo=!0,e.updateOffset(this),lo=!1)}function v(t){return t instanceof m||null!=t&&null!=t._isAMomentObject}function g(t){return 0>t?Math.ceil(t):Math.floor(t)}function y(t){var e=+t,i=0;return 0!==e&&isFinite(e)&&(i=g(e)),i}function b(t,e,i){var o,n=Math.min(t.length,e.length),s=Math.abs(t.length-e.length),r=0;for(o=0;n>o;o++)(i&&t[o]!==e[o]||!i&&y(t[o])!==y(e[o]))&&r++;return r+s}function w(t){e.suppressDeprecationWarnings===!1&&"undefined"!=typeof console&&console.warn&&console.warn("Deprecation warning: "+t)}function _(t,i){var o=!0;return a(function(){return null!=e.deprecationHandler&&e.deprecationHandler(null,t),o&&(w(t+"\nArguments: "+Array.prototype.slice.call(arguments).join(", ")+"\n"+(new Error).stack),o=!1),i.apply(this,arguments)},i)}function x(t,i){null!=e.deprecationHandler&&e.deprecationHandler(t,i),co[t]||(w(i),co[t]=!0)}function k(t){return t instanceof Function||"[object Function]"===Object.prototype.toString.call(t)}function O(t){return"[object Object]"===Object.prototype.toString.call(t)}function M(t){var e,i;for(i in t)e=t[i],k(e)?this[i]=e:this["_"+i]=e;this._config=t,this._ordinalParseLenient=new RegExp(this._ordinalParse.source+"|"+/\d{1,2}/.source)}function D(t,e){var i,o=a({},t);for(i in e)r(e,i)&&(O(t[i])&&O(e[i])?(o[i]={},a(o[i],t[i]),a(o[i],e[i])):null!=e[i]?o[i]=e[i]:delete o[i]);return o}function S(t){null!=t&&this.set(t)}function C(t){return t?t.toLowerCase().replace("_","-"):t}function T(t){for(var e,i,o,n,s=0;s0;){if(o=E(n.slice(0,e).join("-")))return o;if(i&&i.length>=e&&b(n,i,!0)>=e-1)break;e--}s++}return null}function E(e){var i=null;if(!mo[e]&&"undefined"!=typeof t&&t&&t.exports)try{i=po._abbr,!function(){var t=new Error('Cannot find module "./locale"');throw t.code="MODULE_NOT_FOUND",t}(),P(i)}catch(o){}return mo[e]}function P(t,e){var i;return t&&(i=p(e)?R(t):I(t,e),i&&(po=i)),po._abbr}function I(t,e){return null!==e?(e.abbr=t,null!=mo[t]?(x("defineLocaleOverride","use moment.updateLocale(localeName, config) to change an existing locale. moment.defineLocale(localeName, config) should only be used for creating a new locale"),e=D(mo[t]._config,e)):null!=e.parentLocale&&(null!=mo[e.parentLocale]?e=D(mo[e.parentLocale]._config,e):x("parentLocaleUndefined","specified parentLocale is not defined yet")),mo[t]=new S(e),P(t),mo[t]):(delete mo[t],null)}function N(t,e){if(null!=e){var i;null!=mo[t]&&(e=D(mo[t]._config,e)),i=new S(e),i.parentLocale=mo[t],mo[t]=i,P(t)}else null!=mo[t]&&(null!=mo[t].parentLocale?mo[t]=mo[t].parentLocale:null!=mo[t]&&delete mo[t]);return mo[t]}function R(t){var e;if(t&&t._locale&&t._locale._abbr&&(t=t._locale._abbr),!t)return po;if(!o(t)){if(e=E(t))return e;t=[t]}return T(t)}function z(){return uo(mo)}function L(t,e){var i=t.toLowerCase();vo[i]=vo[i+"s"]=vo[e]=t}function A(t){return"string"==typeof t?vo[t]||vo[t.toLowerCase()]:void 0}function B(t){var e,i,o={};for(i in t)r(t,i)&&(e=A(i),e&&(o[e]=t[i]));return o}function F(t,i){return function(o){return null!=o?(H(this,t,o),e.updateOffset(this,i),this):j(this,t)}}function j(t,e){return t.isValid()?t._d["get"+(t._isUTC?"UTC":"")+e]():NaN}function H(t,e,i){t.isValid()&&t._d["set"+(t._isUTC?"UTC":"")+e](i)}function W(t,e){var i;if("object"==typeof t)for(i in t)this.set(i,t[i]);else if(t=A(t),k(this[t]))return this[t](e);return this}function Y(t,e,i){var o=""+Math.abs(t),n=e-o.length,s=t>=0;return(s?i?"+":"":"-")+Math.pow(10,Math.max(0,n)).toString().substr(1)+o}function G(t,e,i,o){var n=o;"string"==typeof o&&(n=function(){return this[o]()}),t&&(wo[t]=n),e&&(wo[e[0]]=function(){return Y(n.apply(this,arguments),e[1],e[2])}),i&&(wo[i]=function(){return this.localeData().ordinal(n.apply(this,arguments),t)})}function V(t){return t.match(/\[[\s\S]/)?t.replace(/^\[|\]$/g,""):t.replace(/\\/g,"")}function U(t){var e,i,o=t.match(go);for(e=0,i=o.length;i>e;e++)wo[o[e]]?o[e]=wo[o[e]]:o[e]=V(o[e]);return function(e){var n,s="";for(n=0;i>n;n++)s+=o[n]instanceof Function?o[n].call(e,t):o[n];return s}}function q(t,e){return t.isValid()?(e=X(e,t.localeData()),bo[e]=bo[e]||U(e),bo[e](t)):t.localeData().invalidDate()}function X(t,e){function i(t){return e.longDateFormat(t)||t}var o=5;for(yo.lastIndex=0;o>=0&&yo.test(t);)t=t.replace(yo,i),yo.lastIndex=0,o-=1;return t}function Z(t,e,i){Bo[t]=k(e)?e:function(t,o){return t&&i?i:e}}function K(t,e){return r(Bo,t)?Bo[t](e._strict,e._locale):new RegExp(J(t))}function J(t){return Q(t.replace("\\","").replace(/\\(\[)|\\(\])|\[([^\]\[]*)\]|\\(.)/g,function(t,e,i,o,n){return e||i||o||n}))}function Q(t){return t.replace(/[-\/\\^$*+?.()|[\]{}]/g,"\\$&")}function $(t,e){var i,o=e;for("string"==typeof t&&(t=[t]),"number"==typeof e&&(o=function(t,i){i[e]=y(t)}),i=0;io;++o)s=h([2e3,o]),this._shortMonthsParse[o]=this.monthsShort(s,"").toLocaleLowerCase(),this._longMonthsParse[o]=this.months(s,"").toLocaleLowerCase();return i?"MMM"===e?(n=fo.call(this._shortMonthsParse,r),-1!==n?n:null):(n=fo.call(this._longMonthsParse,r),-1!==n?n:null):"MMM"===e?(n=fo.call(this._shortMonthsParse,r),-1!==n?n:(n=fo.call(this._longMonthsParse,r),-1!==n?n:null)):(n=fo.call(this._longMonthsParse,r),-1!==n?n:(n=fo.call(this._shortMonthsParse,r),-1!==n?n:null))}function rt(t,e,i){var o,n,s;if(this._monthsParseExact)return st.call(this,t,e,i);for(this._monthsParse||(this._monthsParse=[],this._longMonthsParse=[],this._shortMonthsParse=[]),o=0;12>o;o++){if(n=h([2e3,o]),i&&!this._longMonthsParse[o]&&(this._longMonthsParse[o]=new RegExp("^"+this.months(n,"").replace(".","")+"$","i"),this._shortMonthsParse[o]=new RegExp("^"+this.monthsShort(n,"").replace(".","")+"$","i")),i||this._monthsParse[o]||(s="^"+this.months(n,"")+"|^"+this.monthsShort(n,""),this._monthsParse[o]=new RegExp(s.replace(".",""),"i")),i&&"MMMM"===e&&this._longMonthsParse[o].test(t))return o;if(i&&"MMM"===e&&this._shortMonthsParse[o].test(t))return o;if(!i&&this._monthsParse[o].test(t))return o}}function at(t,e){var i;if(!t.isValid())return t;if("string"==typeof e)if(/^\d+$/.test(e))e=y(e);else if(e=t.localeData().monthsParse(e),"number"!=typeof e)return t;return i=Math.min(t.date(),it(t.year(),e)),t._d["set"+(t._isUTC?"UTC":"")+"Month"](e,i),t}function ht(t){return null!=t?(at(this,t),e.updateOffset(this,!0),this):j(this,"Month")}function dt(){return it(this.year(),this.month())}function lt(t){return this._monthsParseExact?(r(this,"_monthsRegex")||ut.call(this),t?this._monthsShortStrictRegex:this._monthsShortRegex):this._monthsShortStrictRegex&&t?this._monthsShortStrictRegex:this._monthsShortRegex}function ct(t){return this._monthsParseExact?(r(this,"_monthsRegex")||ut.call(this),t?this._monthsStrictRegex:this._monthsRegex):this._monthsStrictRegex&&t?this._monthsStrictRegex:this._monthsRegex}function ut(){function t(t,e){return e.length-t.length}var e,i,o=[],n=[],s=[];for(e=0;12>e;e++)i=h([2e3,e]),o.push(this.monthsShort(i,"")),n.push(this.months(i,"")),s.push(this.months(i,"")),s.push(this.monthsShort(i,""));for(o.sort(t),n.sort(t),s.sort(t),e=0;12>e;e++)o[e]=Q(o[e]),n[e]=Q(n[e]),s[e]=Q(s[e]);this._monthsRegex=new RegExp("^("+s.join("|")+")","i"),this._monthsShortRegex=this._monthsRegex,this._monthsStrictRegex=new RegExp("^("+n.join("|")+")","i"),this._monthsShortStrictRegex=new RegExp("^("+o.join("|")+")","i")}function pt(t){var e,i=t._a;return i&&-2===l(t).overflow&&(e=i[Ho]<0||i[Ho]>11?Ho:i[Wo]<1||i[Wo]>it(i[jo],i[Ho])?Wo:i[Yo]<0||i[Yo]>24||24===i[Yo]&&(0!==i[Go]||0!==i[Vo]||0!==i[Uo])?Yo:i[Go]<0||i[Go]>59?Go:i[Vo]<0||i[Vo]>59?Vo:i[Uo]<0||i[Uo]>999?Uo:-1,l(t)._overflowDayOfYear&&(jo>e||e>Wo)&&(e=Wo),l(t)._overflowWeeks&&-1===e&&(e=qo),l(t)._overflowWeekday&&-1===e&&(e=Xo),l(t).overflow=e),t}function ft(t){var e,i,o,n,s,r,a=t._i,h=tn.exec(a)||en.exec(a);if(h){for(l(t).iso=!0,e=0,i=nn.length;i>e;e++)if(nn[e][1].exec(h[1])){n=nn[e][0],o=nn[e][2]!==!1;break}if(null==n)return void(t._isValid=!1);if(h[3]){for(e=0,i=sn.length;i>e;e++)if(sn[e][1].exec(h[3])){s=(h[2]||" ")+sn[e][0];break}if(null==s)return void(t._isValid=!1)}if(!o&&null!=s)return void(t._isValid=!1);if(h[4]){if(!on.exec(h[4]))return void(t._isValid=!1);r="Z"}t._f=n+(s||"")+(r||""),Tt(t)}else t._isValid=!1}function mt(t){var i=rn.exec(t._i);return null!==i?void(t._d=new Date(+i[1])):(ft(t),void(t._isValid===!1&&(delete t._isValid,e.createFromInputFallback(t))))}function vt(t,e,i,o,n,s,r){var a=new Date(t,e,i,o,n,s,r);return 100>t&&t>=0&&isFinite(a.getFullYear())&&a.setFullYear(t),a}function gt(t){var e=new Date(Date.UTC.apply(null,arguments));return 100>t&&t>=0&&isFinite(e.getUTCFullYear())&&e.setUTCFullYear(t),e}function yt(t){return bt(t)?366:365}function bt(t){return t%4===0&&t%100!==0||t%400===0}function wt(){return bt(this.year())}function _t(t,e,i){var o=7+e-i,n=(7+gt(t,0,o).getUTCDay()-e)%7;return-n+o-1}function xt(t,e,i,o,n){var s,r,a=(7+i-o)%7,h=_t(t,o,n),d=1+7*(e-1)+a+h;return 0>=d?(s=t-1,r=yt(s)+d):d>yt(t)?(s=t+1,r=d-yt(t)):(s=t,r=d),{year:s,dayOfYear:r}}function kt(t,e,i){var o,n,s=_t(t.year(),e,i),r=Math.floor((t.dayOfYear()-s-1)/7)+1;return 1>r?(n=t.year()-1,o=r+Ot(n,e,i)):r>Ot(t.year(),e,i)?(o=r-Ot(t.year(),e,i),n=t.year()+1):(n=t.year(),o=r),{week:o,year:n}}function Ot(t,e,i){var o=_t(t,e,i),n=_t(t+1,e,i);return(yt(t)-o+n)/7}function Mt(t,e,i){return null!=t?t:null!=e?e:i}function Dt(t){var i=new Date(e.now());return t._useUTC?[i.getUTCFullYear(),i.getUTCMonth(),i.getUTCDate()]:[i.getFullYear(),i.getMonth(),i.getDate()]}function St(t){var e,i,o,n,s=[];if(!t._d){for(o=Dt(t),t._w&&null==t._a[Wo]&&null==t._a[Ho]&&Ct(t),t._dayOfYear&&(n=Mt(t._a[jo],o[jo]),t._dayOfYear>yt(n)&&(l(t)._overflowDayOfYear=!0),i=gt(n,0,t._dayOfYear),t._a[Ho]=i.getUTCMonth(),t._a[Wo]=i.getUTCDate()),e=0;3>e&&null==t._a[e];++e)t._a[e]=s[e]=o[e];for(;7>e;e++)t._a[e]=s[e]=null==t._a[e]?2===e?1:0:t._a[e];24===t._a[Yo]&&0===t._a[Go]&&0===t._a[Vo]&&0===t._a[Uo]&&(t._nextDay=!0,t._a[Yo]=0),t._d=(t._useUTC?gt:vt).apply(null,s),null!=t._tzm&&t._d.setUTCMinutes(t._d.getUTCMinutes()-t._tzm),t._nextDay&&(t._a[Yo]=24)}}function Ct(t){var e,i,o,n,s,r,a,h;e=t._w,null!=e.GG||null!=e.W||null!=e.E?(s=1,r=4,i=Mt(e.GG,t._a[jo],kt(At(),1,4).year),o=Mt(e.W,1),n=Mt(e.E,1),(1>n||n>7)&&(h=!0)):(s=t._locale._week.dow,r=t._locale._week.doy,i=Mt(e.gg,t._a[jo],kt(At(),s,r).year),o=Mt(e.w,1),null!=e.d?(n=e.d,(0>n||n>6)&&(h=!0)):null!=e.e?(n=e.e+s,(e.e<0||e.e>6)&&(h=!0)):n=s),1>o||o>Ot(i,s,r)?l(t)._overflowWeeks=!0:null!=h?l(t)._overflowWeekday=!0:(a=xt(i,o,n,s,r),t._a[jo]=a.year,t._dayOfYear=a.dayOfYear)}function Tt(t){if(t._f===e.ISO_8601)return void ft(t);t._a=[],l(t).empty=!0;var i,o,n,s,r,a=""+t._i,h=a.length,d=0;for(n=X(t._f,t._locale).match(go)||[],i=0;i0&&l(t).unusedInput.push(r),a=a.slice(a.indexOf(o)+o.length),d+=o.length),wo[s]?(o?l(t).empty=!1:l(t).unusedTokens.push(s),et(s,o,t)):t._strict&&!o&&l(t).unusedTokens.push(s);l(t).charsLeftOver=h-d,a.length>0&&l(t).unusedInput.push(a),l(t).bigHour===!0&&t._a[Yo]<=12&&t._a[Yo]>0&&(l(t).bigHour=void 0),l(t).parsedDateParts=t._a.slice(0),l(t).meridiem=t._meridiem,t._a[Yo]=Et(t._locale,t._a[Yo],t._meridiem),St(t),pt(t)}function Et(t,e,i){var o;return null==i?e:null!=t.meridiemHour?t.meridiemHour(e,i):null!=t.isPM?(o=t.isPM(i),o&&12>e&&(e+=12),o||12!==e||(e=0),e):e}function Pt(t){var e,i,o,n,s;if(0===t._f.length)return l(t).invalidFormat=!0,void(t._d=new Date(NaN));for(n=0;ns)&&(o=s,i=e));a(t,i||e)}function It(t){if(!t._d){var e=B(t._i);t._a=s([e.year,e.month,e.day||e.date,e.hour,e.minute,e.second,e.millisecond],function(t){return t&&parseInt(t,10)}),St(t)}}function Nt(t){var e=new m(pt(Rt(t)));return e._nextDay&&(e.add(1,"d"),e._nextDay=void 0),e}function Rt(t){var e=t._i,i=t._f;return t._locale=t._locale||R(t._l),null===e||void 0===i&&""===e?u({nullInput:!0}):("string"==typeof e&&(t._i=e=t._locale.preparse(e)),v(e)?new m(pt(e)):(o(i)?Pt(t):i?Tt(t):n(e)?t._d=e:zt(t),c(t)||(t._d=null),t))}function zt(t){var i=t._i;void 0===i?t._d=new Date(e.now()):n(i)?t._d=new Date(i.valueOf()):"string"==typeof i?mt(t):o(i)?(t._a=s(i.slice(0),function(t){return parseInt(t,10)}),St(t)):"object"==typeof i?It(t):"number"==typeof i?t._d=new Date(i):e.createFromInputFallback(t)}function Lt(t,e,i,o,n){var s={};return"boolean"==typeof i&&(o=i,i=void 0),s._isAMomentObject=!0,s._useUTC=s._isUTC=n,s._l=i,s._i=t,s._f=e,s._strict=o,Nt(s)}function At(t,e,i,o){return Lt(t,e,i,o,!1)}function Bt(t,e){var i,n;if(1===e.length&&o(e[0])&&(e=e[0]),!e.length)return At();for(i=e[0],n=1;nt&&(t=-t,i="-"),i+Y(~~(t/60),2)+e+Y(~~t%60,2)})}function Gt(t,e){var i=(e||"").match(t)||[],o=i[i.length-1]||[],n=(o+"").match(cn)||["-",0,0],s=+(60*n[1])+y(n[2]);return"+"===n[0]?s:-s}function Vt(t,i){var o,s;return i._isUTC?(o=i.clone(),s=(v(t)||n(t)?t.valueOf():At(t).valueOf())-o.valueOf(),o._d.setTime(o._d.valueOf()+s),e.updateOffset(o,!1),o):At(t).local()}function Ut(t){return 15*-Math.round(t._d.getTimezoneOffset()/15)}function qt(t,i){var o,n=this._offset||0;return this.isValid()?null!=t?("string"==typeof t?t=Gt(zo,t):Math.abs(t)<16&&(t=60*t),!this._isUTC&&i&&(o=Ut(this)),this._offset=t,this._isUTC=!0,null!=o&&this.add(o,"m"),n!==t&&(!i||this._changeInProgress?le(this,ne(t-n,"m"),1,!1):this._changeInProgress||(this._changeInProgress=!0,e.updateOffset(this,!0),this._changeInProgress=null)),this):this._isUTC?n:Ut(this):null!=t?this:NaN}function Xt(t,e){return null!=t?("string"!=typeof t&&(t=-t),this.utcOffset(t,e),this):-this.utcOffset()}function Zt(t){return this.utcOffset(0,t)}function Kt(t){return this._isUTC&&(this.utcOffset(0,t),this._isUTC=!1,t&&this.subtract(Ut(this),"m")),this}function Jt(){return this._tzm?this.utcOffset(this._tzm):"string"==typeof this._i&&this.utcOffset(Gt(Ro,this._i)),this}function Qt(t){return this.isValid()?(t=t?At(t).utcOffset():0,(this.utcOffset()-t)%60===0):!1}function $t(){return this.utcOffset()>this.clone().month(0).utcOffset()||this.utcOffset()>this.clone().month(5).utcOffset()}function te(){if(!p(this._isDSTShifted))return this._isDSTShifted;var t={};if(f(t,this),t=Rt(t),t._a){var e=t._isUTC?h(t._a):At(t._a);this._isDSTShifted=this.isValid()&&b(t._a,e.toArray())>0}else this._isDSTShifted=!1;return this._isDSTShifted}function ee(){return this.isValid()?!this._isUTC:!1}function ie(){return this.isValid()?this._isUTC:!1}function oe(){return this.isValid()?this._isUTC&&0===this._offset:!1}function ne(t,e){var i,o,n,s=t,a=null;return Wt(t)?s={ms:t._milliseconds,d:t._days,M:t._months}:"number"==typeof t?(s={},e?s[e]=t:s.milliseconds=t):(a=un.exec(t))?(i="-"===a[1]?-1:1,s={y:0,d:y(a[Wo])*i,h:y(a[Yo])*i,m:y(a[Go])*i,s:y(a[Vo])*i,ms:y(a[Uo])*i}):(a=pn.exec(t))?(i="-"===a[1]?-1:1,s={y:se(a[2],i),M:se(a[3],i),w:se(a[4],i),d:se(a[5],i),h:se(a[6],i),m:se(a[7],i),s:se(a[8],i)}):null==s?s={}:"object"==typeof s&&("from"in s||"to"in s)&&(n=ae(At(s.from),At(s.to)),s={},s.ms=n.milliseconds,s.M=n.months),o=new Ht(s),Wt(t)&&r(t,"_locale")&&(o._locale=t._locale),o}function se(t,e){var i=t&&parseFloat(t.replace(",","."));return(isNaN(i)?0:i)*e}function re(t,e){var i={milliseconds:0,months:0};return i.months=e.month()-t.month()+12*(e.year()-t.year()),t.clone().add(i.months,"M").isAfter(e)&&--i.months, +i.milliseconds=+e-+t.clone().add(i.months,"M"),i}function ae(t,e){var i;return t.isValid()&&e.isValid()?(e=Vt(e,t),t.isBefore(e)?i=re(t,e):(i=re(e,t),i.milliseconds=-i.milliseconds,i.months=-i.months),i):{milliseconds:0,months:0}}function he(t){return 0>t?-1*Math.round(-1*t):Math.round(t)}function de(t,e){return function(i,o){var n,s;return null===o||isNaN(+o)||(x(e,"moment()."+e+"(period, number) is deprecated. Please use moment()."+e+"(number, period)."),s=i,i=o,o=s),i="string"==typeof i?+i:i,n=ne(i,o),le(this,n,t),this}}function le(t,i,o,n){var s=i._milliseconds,r=he(i._days),a=he(i._months);t.isValid()&&(n=null==n?!0:n,s&&t._d.setTime(t._d.valueOf()+s*o),r&&H(t,"Date",j(t,"Date")+r*o),a&&at(t,j(t,"Month")+a*o),n&&e.updateOffset(t,r||a))}function ce(t,e){var i=t||At(),o=Vt(i,this).startOf("day"),n=this.diff(o,"days",!0),s=-6>n?"sameElse":-1>n?"lastWeek":0>n?"lastDay":1>n?"sameDay":2>n?"nextDay":7>n?"nextWeek":"sameElse",r=e&&(k(e[s])?e[s]():e[s]);return this.format(r||this.localeData().calendar(s,this,At(i)))}function ue(){return new m(this)}function pe(t,e){var i=v(t)?t:At(t);return this.isValid()&&i.isValid()?(e=A(p(e)?"millisecond":e),"millisecond"===e?this.valueOf()>i.valueOf():i.valueOf()e-s?(i=t.clone().add(n-1,"months"),o=(e-s)/(s-i)):(i=t.clone().add(n+1,"months"),o=(e-s)/(i-s)),-(n+o)||0}function _e(){return this.clone().locale("en").format("ddd MMM DD YYYY HH:mm:ss [GMT]ZZ")}function xe(){var t=this.clone().utc();return 0s&&(e=s),Xe.call(this,t,e,i,o,n))}function Xe(t,e,i,o,n){var s=xt(t,e,i,o,n),r=gt(s.year,0,s.dayOfYear);return this.year(r.getUTCFullYear()),this.month(r.getUTCMonth()),this.date(r.getUTCDate()),this}function Ze(t){return null==t?Math.ceil((this.month()+1)/3):this.month(3*(t-1)+this.month()%3)}function Ke(t){return kt(t,this._week.dow,this._week.doy).week}function Je(){return this._week.dow}function Qe(){return this._week.doy}function $e(t){var e=this.localeData().week(this);return null==t?e:this.add(7*(t-e),"d")}function ti(t){var e=kt(this,1,4).week;return null==t?e:this.add(7*(t-e),"d")}function ei(t,e){return"string"!=typeof t?t:isNaN(t)?(t=e.weekdaysParse(t),"number"==typeof t?t:null):parseInt(t,10)}function ii(t,e){return o(this._weekdays)?this._weekdays[t.day()]:this._weekdays[this._weekdays.isFormat.test(e)?"format":"standalone"][t.day()]}function oi(t){return this._weekdaysShort[t.day()]}function ni(t){return this._weekdaysMin[t.day()]}function si(t,e,i){var o,n,s,r=t.toLocaleLowerCase();if(!this._weekdaysParse)for(this._weekdaysParse=[],this._shortWeekdaysParse=[],this._minWeekdaysParse=[],o=0;7>o;++o)s=h([2e3,1]).day(o),this._minWeekdaysParse[o]=this.weekdaysMin(s,"").toLocaleLowerCase(),this._shortWeekdaysParse[o]=this.weekdaysShort(s,"").toLocaleLowerCase(),this._weekdaysParse[o]=this.weekdays(s,"").toLocaleLowerCase();return i?"dddd"===e?(n=fo.call(this._weekdaysParse,r),-1!==n?n:null):"ddd"===e?(n=fo.call(this._shortWeekdaysParse,r),-1!==n?n:null):(n=fo.call(this._minWeekdaysParse,r),-1!==n?n:null):"dddd"===e?(n=fo.call(this._weekdaysParse,r),-1!==n?n:(n=fo.call(this._shortWeekdaysParse,r),-1!==n?n:(n=fo.call(this._minWeekdaysParse,r),-1!==n?n:null))):"ddd"===e?(n=fo.call(this._shortWeekdaysParse,r),-1!==n?n:(n=fo.call(this._weekdaysParse,r),-1!==n?n:(n=fo.call(this._minWeekdaysParse,r),-1!==n?n:null))):(n=fo.call(this._minWeekdaysParse,r),-1!==n?n:(n=fo.call(this._weekdaysParse,r),-1!==n?n:(n=fo.call(this._shortWeekdaysParse,r),-1!==n?n:null)))}function ri(t,e,i){var o,n,s;if(this._weekdaysParseExact)return si.call(this,t,e,i);for(this._weekdaysParse||(this._weekdaysParse=[],this._minWeekdaysParse=[],this._shortWeekdaysParse=[],this._fullWeekdaysParse=[]),o=0;7>o;o++){if(n=h([2e3,1]).day(o),i&&!this._fullWeekdaysParse[o]&&(this._fullWeekdaysParse[o]=new RegExp("^"+this.weekdays(n,"").replace(".",".?")+"$","i"),this._shortWeekdaysParse[o]=new RegExp("^"+this.weekdaysShort(n,"").replace(".",".?")+"$","i"),this._minWeekdaysParse[o]=new RegExp("^"+this.weekdaysMin(n,"").replace(".",".?")+"$","i")),this._weekdaysParse[o]||(s="^"+this.weekdays(n,"")+"|^"+this.weekdaysShort(n,"")+"|^"+this.weekdaysMin(n,""),this._weekdaysParse[o]=new RegExp(s.replace(".",""),"i")),i&&"dddd"===e&&this._fullWeekdaysParse[o].test(t))return o;if(i&&"ddd"===e&&this._shortWeekdaysParse[o].test(t))return o;if(i&&"dd"===e&&this._minWeekdaysParse[o].test(t))return o;if(!i&&this._weekdaysParse[o].test(t))return o}}function ai(t){if(!this.isValid())return null!=t?this:NaN;var e=this._isUTC?this._d.getUTCDay():this._d.getDay();return null!=t?(t=ei(t,this.localeData()),this.add(t-e,"d")):e}function hi(t){if(!this.isValid())return null!=t?this:NaN;var e=(this.day()+7-this.localeData()._week.dow)%7;return null==t?e:this.add(t-e,"d")}function di(t){return this.isValid()?null==t?this.day()||7:this.day(this.day()%7?t:t-7):null!=t?this:NaN}function li(t){return this._weekdaysParseExact?(r(this,"_weekdaysRegex")||pi.call(this),t?this._weekdaysStrictRegex:this._weekdaysRegex):this._weekdaysStrictRegex&&t?this._weekdaysStrictRegex:this._weekdaysRegex}function ci(t){return this._weekdaysParseExact?(r(this,"_weekdaysRegex")||pi.call(this),t?this._weekdaysShortStrictRegex:this._weekdaysShortRegex):this._weekdaysShortStrictRegex&&t?this._weekdaysShortStrictRegex:this._weekdaysShortRegex}function ui(t){return this._weekdaysParseExact?(r(this,"_weekdaysRegex")||pi.call(this),t?this._weekdaysMinStrictRegex:this._weekdaysMinRegex):this._weekdaysMinStrictRegex&&t?this._weekdaysMinStrictRegex:this._weekdaysMinRegex}function pi(){function t(t,e){return e.length-t.length}var e,i,o,n,s,r=[],a=[],d=[],l=[];for(e=0;7>e;e++)i=h([2e3,1]).day(e),o=this.weekdaysMin(i,""),n=this.weekdaysShort(i,""),s=this.weekdays(i,""),r.push(o),a.push(n),d.push(s),l.push(o),l.push(n),l.push(s);for(r.sort(t),a.sort(t),d.sort(t),l.sort(t),e=0;7>e;e++)a[e]=Q(a[e]),d[e]=Q(d[e]),l[e]=Q(l[e]);this._weekdaysRegex=new RegExp("^("+l.join("|")+")","i"),this._weekdaysShortRegex=this._weekdaysRegex,this._weekdaysMinRegex=this._weekdaysRegex,this._weekdaysStrictRegex=new RegExp("^("+d.join("|")+")","i"),this._weekdaysShortStrictRegex=new RegExp("^("+a.join("|")+")","i"),this._weekdaysMinStrictRegex=new RegExp("^("+r.join("|")+")","i")}function fi(t){var e=Math.round((this.clone().startOf("day")-this.clone().startOf("year"))/864e5)+1;return null==t?e:this.add(t-e,"d")}function mi(){return this.hours()%12||12}function vi(){return this.hours()||24}function gi(t,e){G(t,0,0,function(){return this.localeData().meridiem(this.hours(),this.minutes(),e)})}function yi(t,e){return e._meridiemParse}function bi(t){return"p"===(t+"").toLowerCase().charAt(0)}function wi(t,e,i){return t>11?i?"pm":"PM":i?"am":"AM"}function _i(t,e){e[Uo]=y(1e3*("0."+t))}function xi(){return this._isUTC?"UTC":""}function ki(){return this._isUTC?"Coordinated Universal Time":""}function Oi(t){return At(1e3*t)}function Mi(){return At.apply(null,arguments).parseZone()}function Di(t,e,i){var o=this._calendar[t];return k(o)?o.call(e,i):o}function Si(t){var e=this._longDateFormat[t],i=this._longDateFormat[t.toUpperCase()];return e||!i?e:(this._longDateFormat[t]=i.replace(/MMMM|MM|DD|dddd/g,function(t){return t.slice(1)}),this._longDateFormat[t])}function Ci(){return this._invalidDate}function Ti(t){return this._ordinal.replace("%d",t)}function Ei(t){return t}function Pi(t,e,i,o){var n=this._relativeTime[i];return k(n)?n(t,e,i,o):n.replace(/%d/i,t)}function Ii(t,e){var i=this._relativeTime[t>0?"future":"past"];return k(i)?i(e):i.replace(/%s/i,e)}function Ni(t,e,i,o){var n=R(),s=h().set(o,e);return n[i](s,t)}function Ri(t,e,i){if("number"==typeof t&&(e=t,t=void 0),t=t||"",null!=e)return Ni(t,e,i,"month");var o,n=[];for(o=0;12>o;o++)n[o]=Ni(t,o,i,"month");return n}function zi(t,e,i,o){"boolean"==typeof t?("number"==typeof e&&(i=e,e=void 0),e=e||""):(e=t,i=e,t=!1,"number"==typeof e&&(i=e,e=void 0),e=e||"");var n=R(),s=t?n._week.dow:0;if(null!=i)return Ni(e,(i+s)%7,o,"day");var r,a=[];for(r=0;7>r;r++)a[r]=Ni(e,(r+s)%7,o,"day");return a}function Li(t,e){return Ri(t,e,"months")}function Ai(t,e){return Ri(t,e,"monthsShort")}function Bi(t,e,i){return zi(t,e,i,"weekdays")}function Fi(t,e,i){return zi(t,e,i,"weekdaysShort")}function ji(t,e,i){return zi(t,e,i,"weekdaysMin")}function Hi(){var t=this._data;return this._milliseconds=jn(this._milliseconds),this._days=jn(this._days),this._months=jn(this._months),t.milliseconds=jn(t.milliseconds),t.seconds=jn(t.seconds),t.minutes=jn(t.minutes),t.hours=jn(t.hours),t.months=jn(t.months),t.years=jn(t.years),this}function Wi(t,e,i,o){var n=ne(e,i);return t._milliseconds+=o*n._milliseconds,t._days+=o*n._days,t._months+=o*n._months,t._bubble()}function Yi(t,e){return Wi(this,t,e,1)}function Gi(t,e){return Wi(this,t,e,-1)}function Vi(t){return 0>t?Math.floor(t):Math.ceil(t)}function Ui(){var t,e,i,o,n,s=this._milliseconds,r=this._days,a=this._months,h=this._data;return s>=0&&r>=0&&a>=0||0>=s&&0>=r&&0>=a||(s+=864e5*Vi(Xi(a)+r),r=0,a=0),h.milliseconds=s%1e3,t=g(s/1e3),h.seconds=t%60,e=g(t/60),h.minutes=e%60,i=g(e/60),h.hours=i%24,r+=g(i/24),n=g(qi(r)),a+=n,r-=Vi(Xi(n)),o=g(a/12),a%=12,h.days=r,h.months=a,h.years=o,this}function qi(t){return 4800*t/146097}function Xi(t){return 146097*t/4800}function Zi(t){var e,i,o=this._milliseconds;if(t=A(t),"month"===t||"year"===t)return e=this._days+o/864e5,i=this._months+qi(e),"month"===t?i:i/12;switch(e=this._days+Math.round(Xi(this._months)),t){case"week":return e/7+o/6048e5;case"day":return e+o/864e5;case"hour":return 24*e+o/36e5;case"minute":return 1440*e+o/6e4;case"second":return 86400*e+o/1e3;case"millisecond":return Math.floor(864e5*e)+o;default:throw new Error("Unknown unit "+t)}}function Ki(){return this._milliseconds+864e5*this._days+this._months%12*2592e6+31536e6*y(this._months/12)}function Ji(t){return function(){return this.as(t)}}function Qi(t){return t=A(t),this[t+"s"]()}function $i(t){return function(){return this._data[t]}}function to(){return g(this.days()/7)}function eo(t,e,i,o,n){return n.relativeTime(e||1,!!i,t,o)}function io(t,e,i){var o=ne(t).abs(),n=is(o.as("s")),s=is(o.as("m")),r=is(o.as("h")),a=is(o.as("d")),h=is(o.as("M")),d=is(o.as("y")),l=n=s&&["m"]||s=r&&["h"]||r=a&&["d"]||a=h&&["M"]||h=d&&["y"]||["yy",d];return l[2]=e,l[3]=+t>0,l[4]=i,eo.apply(null,l)}function oo(t,e){return void 0===os[t]?!1:void 0===e?os[t]:(os[t]=e,!0)}function no(t){var e=this.localeData(),i=io(this,!t,e);return t&&(i=e.pastFuture(+this,i)),e.postformat(i)}function so(){var t,e,i,o=ns(this._milliseconds)/1e3,n=ns(this._days),s=ns(this._months);t=g(o/60),e=g(t/60),o%=60,t%=60,i=g(s/12),s%=12;var r=i,a=s,h=n,d=e,l=t,c=o,u=this.asSeconds();return u?(0>u?"-":"")+"P"+(r?r+"Y":"")+(a?a+"M":"")+(h?h+"D":"")+(d||l||c?"T":"")+(d?d+"H":"")+(l?l+"M":"")+(c?c+"S":""):"P0D"}var ro,ao;ao=Array.prototype.some?Array.prototype.some:function(t){for(var e=Object(this),i=e.length>>>0,o=0;i>o;o++)if(o in e&&t.call(this,e[o],o,e))return!0;return!1};var ho=e.momentProperties=[],lo=!1,co={};e.suppressDeprecationWarnings=!1,e.deprecationHandler=null;var uo;uo=Object.keys?Object.keys:function(t){var e,i=[];for(e in t)r(t,e)&&i.push(e);return i};var po,fo,mo={},vo={},go=/(\[[^\[]*\])|(\\)?([Hh]mm(ss)?|Mo|MM?M?M?|Do|DDDo|DD?D?D?|ddd?d?|do?|w[o|w]?|W[o|W]?|Qo?|YYYYYY|YYYYY|YYYY|YY|gg(ggg?)?|GG(GGG?)?|e|E|a|A|hh?|HH?|kk?|mm?|ss?|S{1,9}|x|X|zz?|ZZ?|.)/g,yo=/(\[[^\[]*\])|(\\)?(LTS|LT|LL?L?L?|l{1,4})/g,bo={},wo={},_o=/\d/,xo=/\d\d/,ko=/\d{3}/,Oo=/\d{4}/,Mo=/[+-]?\d{6}/,Do=/\d\d?/,So=/\d\d\d\d?/,Co=/\d\d\d\d\d\d?/,To=/\d{1,3}/,Eo=/\d{1,4}/,Po=/[+-]?\d{1,6}/,Io=/\d+/,No=/[+-]?\d+/,Ro=/Z|[+-]\d\d:?\d\d/gi,zo=/Z|[+-]\d\d(?::?\d\d)?/gi,Lo=/[+-]?\d+(\.\d{1,3})?/,Ao=/[0-9]*['a-z\u00A0-\u05FF\u0700-\uD7FF\uF900-\uFDCF\uFDF0-\uFFEF]+|[\u0600-\u06FF\/]+(\s*?[\u0600-\u06FF]+){1,2}/i,Bo={},Fo={},jo=0,Ho=1,Wo=2,Yo=3,Go=4,Vo=5,Uo=6,qo=7,Xo=8;fo=Array.prototype.indexOf?Array.prototype.indexOf:function(t){var e;for(e=0;e=t?""+t:"+"+t}),G(0,["YY",2],0,function(){return this.year()%100}),G(0,["YYYY",4],0,"year"),G(0,["YYYYY",5],0,"year"),G(0,["YYYYYY",6,!0],0,"year"),L("year","y"),Z("Y",No),Z("YY",Do,xo),Z("YYYY",Eo,Oo),Z("YYYYY",Po,Mo),Z("YYYYYY",Po,Mo),$(["YYYYY","YYYYYY"],jo),$("YYYY",function(t,i){i[jo]=2===t.length?e.parseTwoDigitYear(t):y(t)}),$("YY",function(t,i){i[jo]=e.parseTwoDigitYear(t)}),$("Y",function(t,e){e[jo]=parseInt(t,10)}),e.parseTwoDigitYear=function(t){return y(t)+(y(t)>68?1900:2e3)};var an=F("FullYear",!0);e.ISO_8601=function(){};var hn=_("moment().min is deprecated, use moment.max instead. https://github.com/moment/moment/issues/1548",function(){var t=At.apply(null,arguments);return this.isValid()&&t.isValid()?this>t?this:t:u()}),dn=_("moment().max is deprecated, use moment.min instead. https://github.com/moment/moment/issues/1548",function(){var t=At.apply(null,arguments);return this.isValid()&&t.isValid()?t>this?this:t:u()}),ln=function(){return Date.now?Date.now():+new Date};Yt("Z",":"),Yt("ZZ",""),Z("Z",zo),Z("ZZ",zo),$(["Z","ZZ"],function(t,e,i){i._useUTC=!0,i._tzm=Gt(zo,t)});var cn=/([\+\-]|\d\d)/gi;e.updateOffset=function(){};var un=/^(\-)?(?:(\d*)[. ])?(\d+)\:(\d+)(?:\:(\d+)\.?(\d{3})?\d*)?$/,pn=/^(-)?P(?:(-?[0-9,.]*)Y)?(?:(-?[0-9,.]*)M)?(?:(-?[0-9,.]*)W)?(?:(-?[0-9,.]*)D)?(?:T(?:(-?[0-9,.]*)H)?(?:(-?[0-9,.]*)M)?(?:(-?[0-9,.]*)S)?)?$/;ne.fn=Ht.prototype;var fn=de(1,"add"),mn=de(-1,"subtract");e.defaultFormat="YYYY-MM-DDTHH:mm:ssZ",e.defaultFormatUtc="YYYY-MM-DDTHH:mm:ss[Z]";var vn=_("moment().lang() is deprecated. Instead, use moment().localeData() to get the language configuration. Use moment().locale() to change languages.",function(t){return void 0===t?this.localeData():this.locale(t)});G(0,["gg",2],0,function(){return this.weekYear()%100}),G(0,["GG",2],0,function(){return this.isoWeekYear()%100}),We("gggg","weekYear"),We("ggggg","weekYear"),We("GGGG","isoWeekYear"),We("GGGGG","isoWeekYear"),L("weekYear","gg"),L("isoWeekYear","GG"),Z("G",No),Z("g",No),Z("GG",Do,xo),Z("gg",Do,xo),Z("GGGG",Eo,Oo),Z("gggg",Eo,Oo),Z("GGGGG",Po,Mo),Z("ggggg",Po,Mo),tt(["gggg","ggggg","GGGG","GGGGG"],function(t,e,i,o){e[o.substr(0,2)]=y(t)}),tt(["gg","GG"],function(t,i,o,n){i[n]=e.parseTwoDigitYear(t)}),G("Q",0,"Qo","quarter"),L("quarter","Q"),Z("Q",_o),$("Q",function(t,e){e[Ho]=3*(y(t)-1)}),G("w",["ww",2],"wo","week"),G("W",["WW",2],"Wo","isoWeek"),L("week","w"),L("isoWeek","W"),Z("w",Do),Z("ww",Do,xo),Z("W",Do),Z("WW",Do,xo),tt(["w","ww","W","WW"],function(t,e,i,o){e[o.substr(0,1)]=y(t)});var gn={dow:0,doy:6};G("D",["DD",2],"Do","date"),L("date","D"),Z("D",Do),Z("DD",Do,xo),Z("Do",function(t,e){return t?e._ordinalParse:e._ordinalParseLenient}),$(["D","DD"],Wo),$("Do",function(t,e){e[Wo]=y(t.match(Do)[0],10)});var yn=F("Date",!0);G("d",0,"do","day"),G("dd",0,0,function(t){return this.localeData().weekdaysMin(this,t)}),G("ddd",0,0,function(t){return this.localeData().weekdaysShort(this,t)}),G("dddd",0,0,function(t){return this.localeData().weekdays(this,t)}),G("e",0,0,"weekday"),G("E",0,0,"isoWeekday"),L("day","d"),L("weekday","e"),L("isoWeekday","E"),Z("d",Do),Z("e",Do),Z("E",Do),Z("dd",function(t,e){return e.weekdaysMinRegex(t)}),Z("ddd",function(t,e){return e.weekdaysShortRegex(t)}),Z("dddd",function(t,e){return e.weekdaysRegex(t)}),tt(["dd","ddd","dddd"],function(t,e,i,o){var n=i._locale.weekdaysParse(t,o,i._strict);null!=n?e.d=n:l(i).invalidWeekday=t}),tt(["d","e","E"],function(t,e,i,o){e[o]=y(t)});var bn="Sunday_Monday_Tuesday_Wednesday_Thursday_Friday_Saturday".split("_"),wn="Sun_Mon_Tue_Wed_Thu_Fri_Sat".split("_"),_n="Su_Mo_Tu_We_Th_Fr_Sa".split("_"),xn=Ao,kn=Ao,On=Ao;G("DDD",["DDDD",3],"DDDo","dayOfYear"),L("dayOfYear","DDD"),Z("DDD",To),Z("DDDD",ko),$(["DDD","DDDD"],function(t,e,i){i._dayOfYear=y(t)}),G("H",["HH",2],0,"hour"),G("h",["hh",2],0,mi),G("k",["kk",2],0,vi),G("hmm",0,0,function(){return""+mi.apply(this)+Y(this.minutes(),2)}),G("hmmss",0,0,function(){return""+mi.apply(this)+Y(this.minutes(),2)+Y(this.seconds(),2)}),G("Hmm",0,0,function(){return""+this.hours()+Y(this.minutes(),2)}),G("Hmmss",0,0,function(){return""+this.hours()+Y(this.minutes(),2)+Y(this.seconds(),2)}),gi("a",!0),gi("A",!1),L("hour","h"),Z("a",yi),Z("A",yi),Z("H",Do),Z("h",Do),Z("HH",Do,xo),Z("hh",Do,xo),Z("hmm",So),Z("hmmss",Co),Z("Hmm",So),Z("Hmmss",Co),$(["H","HH"],Yo),$(["a","A"],function(t,e,i){i._isPm=i._locale.isPM(t),i._meridiem=t}),$(["h","hh"],function(t,e,i){e[Yo]=y(t),l(i).bigHour=!0}),$("hmm",function(t,e,i){var o=t.length-2;e[Yo]=y(t.substr(0,o)),e[Go]=y(t.substr(o)),l(i).bigHour=!0}),$("hmmss",function(t,e,i){var o=t.length-4,n=t.length-2;e[Yo]=y(t.substr(0,o)),e[Go]=y(t.substr(o,2)),e[Vo]=y(t.substr(n)),l(i).bigHour=!0}),$("Hmm",function(t,e,i){var o=t.length-2;e[Yo]=y(t.substr(0,o)),e[Go]=y(t.substr(o))}),$("Hmmss",function(t,e,i){var o=t.length-4,n=t.length-2;e[Yo]=y(t.substr(0,o)),e[Go]=y(t.substr(o,2)),e[Vo]=y(t.substr(n))});var Mn=/[ap]\.?m?\.?/i,Dn=F("Hours",!0);G("m",["mm",2],0,"minute"),L("minute","m"),Z("m",Do),Z("mm",Do,xo),$(["m","mm"],Go);var Sn=F("Minutes",!1);G("s",["ss",2],0,"second"),L("second","s"),Z("s",Do),Z("ss",Do,xo),$(["s","ss"],Vo);var Cn=F("Seconds",!1);G("S",0,0,function(){return~~(this.millisecond()/100)}),G(0,["SS",2],0,function(){return~~(this.millisecond()/10)}),G(0,["SSS",3],0,"millisecond"),G(0,["SSSS",4],0,function(){return 10*this.millisecond()}),G(0,["SSSSS",5],0,function(){return 100*this.millisecond()}),G(0,["SSSSSS",6],0,function(){return 1e3*this.millisecond()}),G(0,["SSSSSSS",7],0,function(){return 1e4*this.millisecond()}),G(0,["SSSSSSSS",8],0,function(){return 1e5*this.millisecond()}),G(0,["SSSSSSSSS",9],0,function(){return 1e6*this.millisecond()}),L("millisecond","ms"),Z("S",To,_o),Z("SS",To,xo),Z("SSS",To,ko);var Tn;for(Tn="SSSS";Tn.length<=9;Tn+="S")Z(Tn,Io);for(Tn="S";Tn.length<=9;Tn+="S")$(Tn,_i);var En=F("Milliseconds",!1);G("z",0,0,"zoneAbbr"),G("zz",0,0,"zoneName");var Pn=m.prototype;Pn.add=fn,Pn.calendar=ce,Pn.clone=ue,Pn.diff=be,Pn.endOf=Pe,Pn.format=ke,Pn.from=Oe,Pn.fromNow=Me,Pn.to=De,Pn.toNow=Se,Pn.get=W,Pn.invalidAt=je,Pn.isAfter=pe,Pn.isBefore=fe,Pn.isBetween=me,Pn.isSame=ve,Pn.isSameOrAfter=ge,Pn.isSameOrBefore=ye,Pn.isValid=Be,Pn.lang=vn,Pn.locale=Ce,Pn.localeData=Te,Pn.max=dn,Pn.min=hn,Pn.parsingFlags=Fe,Pn.set=W,Pn.startOf=Ee,Pn.subtract=mn,Pn.toArray=ze,Pn.toObject=Le,Pn.toDate=Re,Pn.toISOString=xe,Pn.toJSON=Ae,Pn.toString=_e,Pn.unix=Ne,Pn.valueOf=Ie,Pn.creationData=He,Pn.year=an,Pn.isLeapYear=wt,Pn.weekYear=Ye,Pn.isoWeekYear=Ge,Pn.quarter=Pn.quarters=Ze,Pn.month=ht,Pn.daysInMonth=dt,Pn.week=Pn.weeks=$e,Pn.isoWeek=Pn.isoWeeks=ti,Pn.weeksInYear=Ue,Pn.isoWeeksInYear=Ve,Pn.date=yn,Pn.day=Pn.days=ai,Pn.weekday=hi,Pn.isoWeekday=di,Pn.dayOfYear=fi,Pn.hour=Pn.hours=Dn,Pn.minute=Pn.minutes=Sn,Pn.second=Pn.seconds=Cn,Pn.millisecond=Pn.milliseconds=En,Pn.utcOffset=qt,Pn.utc=Zt,Pn.local=Kt,Pn.parseZone=Jt,Pn.hasAlignedHourOffset=Qt,Pn.isDST=$t,Pn.isDSTShifted=te,Pn.isLocal=ee,Pn.isUtcOffset=ie,Pn.isUtc=oe,Pn.isUTC=oe,Pn.zoneAbbr=xi,Pn.zoneName=ki,Pn.dates=_("dates accessor is deprecated. Use date instead.",yn),Pn.months=_("months accessor is deprecated. Use month instead",ht),Pn.years=_("years accessor is deprecated. Use year instead",an),Pn.zone=_("moment().zone is deprecated, use moment().utcOffset instead. https://github.com/moment/moment/issues/1779",Xt);var In=Pn,Nn={sameDay:"[Today at] LT",nextDay:"[Tomorrow at] LT",nextWeek:"dddd [at] LT",lastDay:"[Yesterday at] LT",lastWeek:"[Last] dddd [at] LT",sameElse:"L"},Rn={LTS:"h:mm:ss A",LT:"h:mm A",L:"MM/DD/YYYY",LL:"MMMM D, YYYY",LLL:"MMMM D, YYYY h:mm A",LLLL:"dddd, MMMM D, YYYY h:mm A"},zn="Invalid date",Ln="%d",An=/\d{1,2}/,Bn={future:"in %s",past:"%s ago",s:"a few seconds",m:"a minute",mm:"%d minutes",h:"an hour",hh:"%d hours",d:"a day",dd:"%d days",M:"a month",MM:"%d months",y:"a year",yy:"%d years"},Fn=S.prototype;Fn._calendar=Nn,Fn.calendar=Di,Fn._longDateFormat=Rn,Fn.longDateFormat=Si,Fn._invalidDate=zn,Fn.invalidDate=Ci,Fn._ordinal=Ln,Fn.ordinal=Ti,Fn._ordinalParse=An,Fn.preparse=Ei,Fn.postformat=Ei,Fn._relativeTime=Bn,Fn.relativeTime=Pi,Fn.pastFuture=Ii,Fn.set=M,Fn.months=ot,Fn._months=Ko,Fn.monthsShort=nt,Fn._monthsShort=Jo,Fn.monthsParse=rt,Fn._monthsRegex=$o,Fn.monthsRegex=ct,Fn._monthsShortRegex=Qo,Fn.monthsShortRegex=lt,Fn.week=Ke,Fn._week=gn,Fn.firstDayOfYear=Qe,Fn.firstDayOfWeek=Je,Fn.weekdays=ii,Fn._weekdays=bn,Fn.weekdaysMin=ni,Fn._weekdaysMin=_n,Fn.weekdaysShort=oi,Fn._weekdaysShort=wn,Fn.weekdaysParse=ri,Fn._weekdaysRegex=xn,Fn.weekdaysRegex=li,Fn._weekdaysShortRegex=kn,Fn.weekdaysShortRegex=ci,Fn._weekdaysMinRegex=On,Fn.weekdaysMinRegex=ui,Fn.isPM=bi,Fn._meridiemParse=Mn,Fn.meridiem=wi,P("en",{ordinalParse:/\d{1,2}(th|st|nd|rd)/,ordinal:function(t){var e=t%10,i=1===y(t%100/10)?"th":1===e?"st":2===e?"nd":3===e?"rd":"th";return t+i}}),e.lang=_("moment.lang is deprecated. Use moment.locale instead.",P),e.langData=_("moment.langData is deprecated. Use moment.localeData instead.",R);var jn=Math.abs,Hn=Ji("ms"),Wn=Ji("s"),Yn=Ji("m"),Gn=Ji("h"),Vn=Ji("d"),Un=Ji("w"),qn=Ji("M"),Xn=Ji("y"),Zn=$i("milliseconds"),Kn=$i("seconds"),Jn=$i("minutes"),Qn=$i("hours"),$n=$i("days"),ts=$i("months"),es=$i("years"),is=Math.round,os={s:45,m:45,h:22,d:26,M:11},ns=Math.abs,ss=Ht.prototype;ss.abs=Hi,ss.add=Yi,ss.subtract=Gi,ss.as=Zi,ss.asMilliseconds=Hn,ss.asSeconds=Wn,ss.asMinutes=Yn,ss.asHours=Gn,ss.asDays=Vn,ss.asWeeks=Un,ss.asMonths=qn,ss.asYears=Xn,ss.valueOf=Ki,ss._bubble=Ui,ss.get=Qi,ss.milliseconds=Zn,ss.seconds=Kn,ss.minutes=Jn,ss.hours=Qn,ss.days=$n,ss.weeks=to,ss.months=ts,ss.years=es,ss.humanize=no,ss.toISOString=so,ss.toString=so,ss.toJSON=so,ss.locale=Ce,ss.localeData=Te,ss.toIsoString=_("toIsoString() is deprecated. Please use toISOString() instead (notice the capitals)",so),ss.lang=vn,G("X",0,0,"unix"),G("x",0,0,"valueOf"),Z("x",No),Z("X",Lo),$("X",function(t,e,i){i._d=new Date(1e3*parseFloat(t,10))}),$("x",function(t,e,i){i._d=new Date(y(t))}),e.version="2.13.0",i(At),e.fn=In,e.min=Ft,e.max=jt,e.now=ln,e.utc=h,e.unix=Oi,e.months=Li,e.isDate=n,e.locale=P,e.invalid=u,e.duration=ne,e.isMoment=v,e.weekdays=Bi,e.parseZone=Mi,e.localeData=R,e.isDuration=Wt,e.monthsShort=Ai,e.weekdaysMin=ji,e.defineLocale=I,e.updateLocale=N,e.locales=z,e.weekdaysShort=Fi,e.normalizeUnits=A,e.relativeTimeThreshold=oo,e.prototype=In;var rs=e;return rs})}).call(e,i(4)(t))},function(t,e){t.exports=function(t){return t.webpackPolyfill||(t.deprecate=function(){},t.paths=[],t.children=[],t.webpackPolyfill=1),t}},function(t,e){function i(t){throw new Error("Cannot find module '"+t+"'.")}i.keys=function(){return[]},i.resolve=i,t.exports=i,i.id=5},function(t,e){(function(e){function i(t,e,i){var o=e&&i||0,n=0;for(e=e||[],t.toLowerCase().replace(/[0-9a-f]{2}/g,function(t){16>n&&(e[o+n++]=c[t])});16>n;)e[o+n++]=0;return e}function o(t,e){var i=e||0,o=l;return o[t[i++]]+o[t[i++]]+o[t[i++]]+o[t[i++]]+"-"+o[t[i++]]+o[t[i++]]+"-"+o[t[i++]]+o[t[i++]]+"-"+o[t[i++]]+o[t[i++]]+"-"+o[t[i++]]+o[t[i++]]+o[t[i++]]+o[t[i++]]+o[t[i++]]+o[t[i++]]}function n(t,e,i){var n=e&&i||0,s=e||[];t=t||{};var r=void 0!==t.clockseq?t.clockseq:m,a=void 0!==t.msecs?t.msecs:(new Date).getTime(),h=void 0!==t.nsecs?t.nsecs:g+1,d=a-v+(h-g)/1e4;if(0>d&&void 0===t.clockseq&&(r=r+1&16383),(0>d||a>v)&&void 0===t.nsecs&&(h=0),h>=1e4)throw new Error("uuid.v1(): Can't create more than 10M uuids/sec");v=a,g=h,m=r,a+=122192928e5;var l=(1e4*(268435455&a)+h)%4294967296;s[n++]=l>>>24&255,s[n++]=l>>>16&255,s[n++]=l>>>8&255,s[n++]=255&l;var c=a/4294967296*1e4&268435455;s[n++]=c>>>8&255,s[n++]=255&c,s[n++]=c>>>24&15|16,s[n++]=c>>>16&255,s[n++]=r>>>8|128,s[n++]=255&r;for(var u=t.node||f,p=0;6>p;p++)s[n+p]=u[p];return e?e:o(s)}function s(t,e,i){var n=e&&i||0;"string"==typeof t&&(e="binary"==t?new Array(16):null,t=null),t=t||{};var s=t.random||(t.rng||r)();if(s[6]=15&s[6]|64,s[8]=63&s[8]|128,e)for(var a=0;16>a;a++)e[n+a]=s[a];return e||o(s)}var r,a="undefined"!=typeof window?window:"undefined"!=typeof e?e:null;if(a&&a.crypto&&crypto.getRandomValues){var h=new Uint8Array(16);r=function(){return crypto.getRandomValues(h),h}}if(!r){var d=new Array(16);r=function(){for(var t,e=0;16>e;e++)0===(3&e)&&(t=4294967296*Math.random()),d[e]=t>>>((3&e)<<3)&255;return d}}for(var l=[],c={},u=0;256>u;u++)l[u]=(u+256).toString(16).substr(1),c[l[u]]=u;var p=r(),f=[1|p[0],p[1],p[2],p[3],p[4],p[5]],m=16383&(p[6]<<8|p[7]),v=0,g=0,y=s;y.v1=n,y.v4=s,y.parse=i,y.unparse=o,t.exports=y}).call(e,function(){return this}())},function(t,e,i){e.util=i(1),e.DOMutil=i(8),e.DataSet=i(9),e.DataView=i(11),e.Queue=i(10),e.Graph3d=i(12),e.graph3d={Camera:i(16),Filter:i(17),Point2d:i(15),Point3d:i(14),Slider:i(18),StepNumber:i(19)},e.moment=i(2),e.Hammer=i(20),e.keycharm=i(23)},function(t,e){e.prepareElements=function(t){for(var e in t)t.hasOwnProperty(e)&&(t[e].redundant=t[e].used,t[e].used=[])},e.cleanupElements=function(t){for(var e in t)if(t.hasOwnProperty(e)&&t[e].redundant){for(var i=0;i0?(o=e[t].redundant[0],e[t].redundant.shift()):(o=document.createElementNS("http://www.w3.org/2000/svg",t),i.appendChild(o)):(o=document.createElementNS("http://www.w3.org/2000/svg",t),e[t]={used:[],redundant:[]},i.appendChild(o)),e[t].used.push(o),o},e.getDOMElement=function(t,e,i,o){var n;return e.hasOwnProperty(t)?e[t].redundant.length>0?(n=e[t].redundant[0],e[t].redundant.shift()):(n=document.createElement(t),void 0!==o?i.insertBefore(n,o):i.appendChild(n)):(n=document.createElement(t),e[t]={used:[],redundant:[]},void 0!==o?i.insertBefore(n,o):i.appendChild(n)),e[t].used.push(n),n},e.drawPoint=function(t,i,o,n,s,r){var a;if("circle"==o.style?(a=e.getSVGElement("circle",n,s),a.setAttributeNS(null,"cx",t),a.setAttributeNS(null,"cy",i),a.setAttributeNS(null,"r",.5*o.size)):(a=e.getSVGElement("rect",n,s),a.setAttributeNS(null,"x",t-.5*o.size),a.setAttributeNS(null,"y",i-.5*o.size),a.setAttributeNS(null,"width",o.size),a.setAttributeNS(null,"height",o.size)),void 0!==o.styles&&a.setAttributeNS(null,"style",o.styles),a.setAttributeNS(null,"class",o.className+" vis-point"),r){var h=e.getSVGElement("text",n,s); +r.xOffset&&(t+=r.xOffset),r.yOffset&&(i+=r.yOffset),r.content&&(h.textContent=r.content),r.className&&h.setAttributeNS(null,"class",r.className+" vis-label"),h.setAttributeNS(null,"x",t),h.setAttributeNS(null,"y",i)}return a},e.drawBar=function(t,i,o,n,s,r,a,h){if(0!=n){0>n&&(n*=-1,i-=n);var d=e.getSVGElement("rect",r,a);d.setAttributeNS(null,"x",t-.5*o),d.setAttributeNS(null,"y",i),d.setAttributeNS(null,"width",o),d.setAttributeNS(null,"height",n),d.setAttributeNS(null,"class",s),h&&d.setAttributeNS(null,"style",h)}}},function(t,e,i){function o(t,e){if(t&&!Array.isArray(t)&&(e=t,t=null),this._options=e||{},this._data={},this.length=0,this._fieldId=this._options.fieldId||"id",this._type={},this._options.type)for(var i=Object.keys(this._options.type),o=0,n=i.length;n>o;o++){var s=i[o],r=this._options.type[s];"Date"==r||"ISODate"==r||"ASPDate"==r?this._type[s]="Date":this._type[s]=r}if(this._options.convert)throw new Error('Option "convert" is deprecated. Use "type" instead.');this._subscribers={},t&&this.add(t),this.setOptions(e)}var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=i(1),r=i(10);o.prototype.setOptions=function(t){t&&void 0!==t.queue&&(t.queue===!1?this._queue&&(this._queue.destroy(),delete this._queue):(this._queue||(this._queue=r.extend(this,{replace:["add","update","remove"]})),"object"===n(t.queue)&&this._queue.setOptions(t.queue)))},o.prototype.on=function(t,e){var i=this._subscribers[t];i||(i=[],this._subscribers[t]=i),i.push({callback:e})},o.prototype.subscribe=function(){throw new Error("DataSet.subscribe is deprecated. Use DataSet.on instead.")},o.prototype.off=function(t,e){var i=this._subscribers[t];i&&(this._subscribers[t]=i.filter(function(t){return t.callback!=e}))},o.prototype.unsubscribe=function(){throw new Error("DataSet.unsubscribe is deprecated. Use DataSet.off instead.")},o.prototype._trigger=function(t,e,i){if("*"==t)throw new Error("Cannot trigger event *");var o=[];t in this._subscribers&&(o=o.concat(this._subscribers[t])),"*"in this._subscribers&&(o=o.concat(this._subscribers["*"]));for(var n=0,s=o.length;s>n;n++){var r=o[n];r.callback&&r.callback(t,e,i||null)}},o.prototype.add=function(t,e){var i,o=[],n=this;if(Array.isArray(t))for(var s=0,r=t.length;r>s;s++)i=n._addItem(t[s]),o.push(i);else{if(!(t instanceof Object))throw new Error("Unknown dataType");i=n._addItem(t),o.push(i)}return o.length&&this._trigger("add",{items:o},e),o},o.prototype.update=function(t,e){var i=[],o=[],n=[],r=[],a=this,h=a._fieldId,d=function(t){var e=t[h];if(a._data[e]){var d=s.extend({},a._data[e]);e=a._updateItem(t),o.push(e),r.push(t),n.push(d)}else e=a._addItem(t),i.push(e)};if(Array.isArray(t))for(var l=0,c=t.length;c>l;l++)t[l]instanceof Object?d(t[l]):console.warn("Ignoring input item, which is not an object at index "+l);else{if(!(t instanceof Object))throw new Error("Unknown dataType");d(t)}if(i.length&&this._trigger("add",{items:i},e),o.length){var u={items:o,oldData:n,data:r};this._trigger("update",u,e)}return i.concat(o)},o.prototype.get=function(t){var e,i,o,n=this,r=s.getType(arguments[0]);"String"==r||"Number"==r?(e=arguments[0],o=arguments[1]):"Array"==r?(i=arguments[0],o=arguments[1]):o=arguments[0];var a;if(o&&o.returnType){var h=["Array","Object"];a=-1==h.indexOf(o.returnType)?"Array":o.returnType}else a="Array";var d,l,c,u,p,f=o&&o.type||this._options.type,m=o&&o.filter,v=[];if(void 0!=e)d=n._getItem(e,f),d&&m&&!m(d)&&(d=null);else if(void 0!=i)for(u=0,p=i.length;p>u;u++)d=n._getItem(i[u],f),m&&!m(d)||v.push(d);else for(l=Object.keys(this._data),u=0,p=l.length;p>u;u++)c=l[u],d=n._getItem(c,f),m&&!m(d)||v.push(d);if(o&&o.order&&void 0==e&&this._sort(v,o.order),o&&o.fields){var g=o.fields;if(void 0!=e)d=this._filterFields(d,g);else for(u=0,p=v.length;p>u;u++)v[u]=this._filterFields(v[u],g)}if("Object"==a){var y,b={};for(u=0,p=v.length;p>u;u++)y=v[u],b[y.id]=y;return b}return void 0!=e?d:v},o.prototype.getIds=function(t){var e,i,o,n,s,r=this._data,a=t&&t.filter,h=t&&t.order,d=t&&t.type||this._options.type,l=Object.keys(r),c=[];if(a)if(h){for(s=[],e=0,i=l.length;i>e;e++)o=l[e],n=this._getItem(o,d),a(n)&&s.push(n);for(this._sort(s,h),e=0,i=s.length;i>e;e++)c.push(s[e][this._fieldId])}else for(e=0,i=l.length;i>e;e++)o=l[e],n=this._getItem(o,d),a(n)&&c.push(n[this._fieldId]);else if(h){for(s=[],e=0,i=l.length;i>e;e++)o=l[e],s.push(r[o]);for(this._sort(s,h),e=0,i=s.length;i>e;e++)c.push(s[e][this._fieldId])}else for(e=0,i=l.length;i>e;e++)o=l[e],n=r[o],c.push(n[this._fieldId]);return c},o.prototype.getDataSet=function(){return this},o.prototype.forEach=function(t,e){var i,o,n,s,r=e&&e.filter,a=e&&e.type||this._options.type,h=this._data,d=Object.keys(h);if(e&&e.order){var l=this.get(e);for(i=0,o=l.length;o>i;i++)n=l[i],s=n[this._fieldId],t(n,s)}else for(i=0,o=d.length;o>i;i++)s=d[i],n=this._getItem(s,a),r&&!r(n)||t(n,s)},o.prototype.map=function(t,e){var i,o,n,s,r=e&&e.filter,a=e&&e.type||this._options.type,h=[],d=this._data,l=Object.keys(d);for(i=0,o=l.length;o>i;i++)n=l[i],s=this._getItem(n,a),r&&!r(s)||h.push(t(s,n));return e&&e.order&&this._sort(h,e.order),h},o.prototype._filterFields=function(t,e){if(!t)return t;var i,o,n={},s=Object.keys(t),r=s.length;if(Array.isArray(e))for(i=0;r>i;i++)o=s[i],-1!=e.indexOf(o)&&(n[o]=t[o]);else for(i=0;r>i;i++)o=s[i],e.hasOwnProperty(o)&&(n[e[o]]=t[o]);return n},o.prototype._sort=function(t,e){if(s.isString(e)){var i=e;t.sort(function(t,e){var o=t[i],n=e[i];return o>n?1:n>o?-1:0})}else{if("function"!=typeof e)throw new TypeError("Order must be a function or a string");t.sort(e)}},o.prototype.remove=function(t,e){var i,o,n,s=[];if(Array.isArray(t))for(i=0,o=t.length;o>i;i++)n=this._remove(t[i]),null!=n&&s.push(n);else n=this._remove(t),null!=n&&s.push(n);return s.length&&this._trigger("remove",{items:s},e),s},o.prototype._remove=function(t){if(s.isNumber(t)||s.isString(t)){if(this._data[t])return delete this._data[t],this.length--,t}else if(t instanceof Object){var e=t[this._fieldId];if(void 0!==e&&this._data[e])return delete this._data[e],this.length--,e}return null},o.prototype.clear=function(t){var e=Object.keys(this._data);return this._data={},this.length=0,this._trigger("remove",{items:e},t),e},o.prototype.max=function(t){var e,i,o=this._data,n=Object.keys(o),s=null,r=null;for(e=0,i=n.length;i>e;e++){var a=n[e],h=o[a],d=h[t];null!=d&&(!s||d>r)&&(s=h,r=d)}return s},o.prototype.min=function(t){var e,i,o=this._data,n=Object.keys(o),s=null,r=null;for(e=0,i=n.length;i>e;e++){var a=n[e],h=o[a],d=h[t];null!=d&&(!s||r>d)&&(s=h,r=d)}return s},o.prototype.distinct=function(t){var e,i,o,n=this._data,r=Object.keys(n),a=[],h=this._options.type&&this._options.type[t]||null,d=0;for(e=0,o=r.length;o>e;e++){var l=r[e],c=n[l],u=c[t],p=!1;for(i=0;d>i;i++)if(a[i]==u){p=!0;break}p||void 0===u||(a[d]=u,d++)}if(h)for(e=0,o=a.length;o>e;e++)a[e]=s.convert(a[e],h);return a},o.prototype._addItem=function(t){var e=t[this._fieldId];if(void 0!=e){if(this._data[e])throw new Error("Cannot add item: item with id "+e+" already exists")}else e=s.randomUUID(),t[this._fieldId]=e;var i,o,n={},r=Object.keys(t);for(i=0,o=r.length;o>i;i++){var a=r[i],h=this._type[a];n[a]=s.convert(t[a],h)}return this._data[e]=n,this.length++,e},o.prototype._getItem=function(t,e){var i,o,n,r,a=this._data[t];if(!a)return null;var h={},d=Object.keys(a);if(e)for(n=0,r=d.length;r>n;n++)i=d[n],o=a[i],h[i]=s.convert(o,e[i]);else for(n=0,r=d.length;r>n;n++)i=d[n],o=a[i],h[i]=o;return h},o.prototype._updateItem=function(t){var e=t[this._fieldId];if(void 0==e)throw new Error("Cannot update item: item has no id (item: "+JSON.stringify(t)+")");var i=this._data[e];if(!i)throw new Error("Cannot update item: no item with id "+e+" found");for(var o=Object.keys(t),n=0,r=o.length;r>n;n++){var a=o[n],h=this._type[a];i[a]=s.convert(t[a],h)}return e},t.exports=o},function(t,e){function i(t){this.delay=null,this.max=1/0,this._queue=[],this._timeout=null,this._extended=null,this.setOptions(t)}i.prototype.setOptions=function(t){t&&"undefined"!=typeof t.delay&&(this.delay=t.delay),t&&"undefined"!=typeof t.max&&(this.max=t.max),this._flushIfNeeded()},i.extend=function(t,e){var o=new i(e);if(void 0!==t.flush)throw new Error("Target object already has a property flush");t.flush=function(){o.flush()};var n=[{name:"flush",original:void 0}];if(e&&e.replace)for(var s=0;sthis.max&&this.flush(),clearTimeout(this._timeout),this.queue.length>0&&"number"==typeof this.delay){var t=this;this._timeout=setTimeout(function(){t.flush()},this.delay)}},i.prototype.flush=function(){for(;this._queue.length>0;){var t=this._queue.shift();t.fn.apply(t.context||t.fn,t.args||[])}},t.exports=i},function(t,e,i){function o(t,e){this._data=null,this._ids={},this.length=0,this._options=e||{},this._fieldId="id",this._subscribers={};var i=this;this.listener=function(){i._onEvent.apply(i,arguments)},this.setData(t)}var n=i(1),s=i(9);o.prototype.setData=function(t){var e,i,o,n;if(this._data&&(this._data.off&&this._data.off("*",this.listener),e=Object.keys(this._ids),this._ids={},this.length=0,this._trigger("remove",{items:e})),this._data=t,this._data){for(this._fieldId=this._options.fieldId||this._data&&this._data.options&&this._data.options.fieldId||"id",e=this._data.getIds({filter:this._options&&this._options.filter}),o=0,n=e.length;n>o;o++)i=e[o],this._ids[i]=!0;this.length=e.length,this._trigger("add",{items:e}),this._data.on&&this._data.on("*",this.listener)}},o.prototype.refresh=function(){var t,e,i,o=this._data.getIds({filter:this._options&&this._options.filter}),n=Object.keys(this._ids),s={},r=[],a=[];for(e=0,i=o.length;i>e;e++)t=o[e],s[t]=!0,this._ids[t]||(r.push(t),this._ids[t]=!0);for(e=0,i=n.length;i>e;e++)t=n[e],s[t]||(a.push(t),delete this._ids[t]);this.length+=r.length-a.length,r.length&&this._trigger("add",{items:r}),a.length&&this._trigger("remove",{items:a})},o.prototype.get=function(t){var e,i,o,s=this,r=n.getType(arguments[0]);"String"==r||"Number"==r||"Array"==r?(e=arguments[0],i=arguments[1],o=arguments[2]):(i=arguments[0],o=arguments[1]);var a=n.extend({},this._options,i);this._options.filter&&i&&i.filter&&(a.filter=function(t){return s._options.filter(t)&&i.filter(t)});var h=[];return void 0!=e&&h.push(e),h.push(a),h.push(o),this._data&&this._data.get.apply(this._data,h)},o.prototype.getIds=function(t){var e;if(this._data){var i,o=this._options.filter;i=t&&t.filter?o?function(e){return o(e)&&t.filter(e)}:t.filter:o,e=this._data.getIds({filter:i,order:t&&t.order})}else e=[];return e},o.prototype.map=function(t,e){var i=[];if(this._data){var o,n=this._options.filter;o=e&&e.filter?n?function(t){return n(t)&&e.filter(t)}:e.filter:n,i=this._data.map(t,{filter:o,order:e&&e.order})}else i=[];return i},o.prototype.getDataSet=function(){for(var t=this;t instanceof o;)t=t._data;return t||null},o.prototype._onEvent=function(t,e,i){var o,n,s,r,a=e&&e.items,h=this._data,d=[],l=[],c=[],u=[];if(a&&h){switch(t){case"add":for(o=0,n=a.length;n>o;o++)s=a[o],r=this.get(s),r&&(this._ids[s]=!0,l.push(s));break;case"update":for(o=0,n=a.length;n>o;o++)s=a[o],r=this.get(s),r?this._ids[s]?(c.push(s),d.push(e.data[o])):(this._ids[s]=!0,l.push(s)):this._ids[s]&&(delete this._ids[s],u.push(s));break;case"remove":for(o=0,n=a.length;n>o;o++)s=a[o],this._ids[s]&&(delete this._ids[s],u.push(s))}this.length+=l.length-u.length,l.length&&this._trigger("add",{items:l},i),c.length&&this._trigger("update",{items:c,data:d},i),u.length&&this._trigger("remove",{items:u},i)}},o.prototype.on=s.prototype.on,o.prototype.off=s.prototype.off,o.prototype._trigger=s.prototype._trigger,o.prototype.subscribe=o.prototype.on,o.prototype.unsubscribe=o.prototype.off,t.exports=o},function(t,e,i){function o(t,e,i){if(!(this instanceof o))throw new SyntaxError("Constructor must be called with the new operator");this.containerElement=t,this.width="400px",this.height="400px",this.margin=10,this.defaultXCenter="55%",this.defaultYCenter="50%",this.xLabel="x",this.yLabel="y",this.zLabel="z";var n=function(t){return t};this.xValueLabel=n,this.yValueLabel=n,this.zValueLabel=n,this.filterLabel="time",this.legendLabel="value",this.style=o.STYLE.DOT,this.showPerspective=!0,this.showGrid=!0,this.keepAspectRatio=!0,this.showShadow=!1,this.showGrayBottom=!1,this.showTooltip=!1,this.verticalRatio=.5,this.animationInterval=1e3,this.animationPreload=!1,this.camera=new p,this.camera.setArmRotation(1,.5),this.camera.setArmLength(1.7),this.eye=new c(0,0,-1),this.dataTable=null,this.dataPoints=null,this.colX=void 0,this.colY=void 0,this.colZ=void 0,this.colValue=void 0,this.colFilter=void 0,this.xMin=0,this.xStep=void 0,this.xMax=1,this.yMin=0,this.yStep=void 0,this.yMax=1,this.zMin=0,this.zStep=void 0,this.zMax=1,this.valueMin=0,this.valueMax=1,this.xBarWidth=1,this.yBarWidth=1,this.axisColor="#4D4D4D",this.gridColor="#D3D3D3",this.dataColor={fill:"#7DC1FF",stroke:"#3267D2",strokeWidth:1},this.dotSizeRatio=.02,this.create(),this.setOptions(i),e&&this.setData(e)}function n(t){return"clientX"in t?t.clientX:t.targetTouches[0]&&t.targetTouches[0].clientX||0}function s(t){return"clientY"in t?t.clientY:t.targetTouches[0]&&t.targetTouches[0].clientY||0}var r="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},a=i(13),h=i(9),d=i(11),l=i(1),c=i(14),u=i(15),p=i(16),f=i(17),m=i(18),v=i(19);a(o.prototype),o.prototype._setScale=function(){this.scale=new c(1/(this.xMax-this.xMin),1/(this.yMax-this.yMin),1/(this.zMax-this.zMin)),this.keepAspectRatio&&(this.scale.x3&&(this.colFilter=3);else{if(this.style!==o.STYLE.DOTCOLOR&&this.style!==o.STYLE.DOTSIZE&&this.style!==o.STYLE.BARCOLOR&&this.style!==o.STYLE.BARSIZE)throw'Unknown style "'+this.style+'"';this.colX=0,this.colY=1,this.colZ=2,this.colValue=3,t.getNumberOfColumns()>4&&(this.colFilter=4)}},o.prototype.getNumberOfRows=function(t){return t.length},o.prototype.getNumberOfColumns=function(t){var e=0;for(var i in t[0])t[0].hasOwnProperty(i)&&e++;return e},o.prototype.getDistinctValues=function(t,e){for(var i=[],o=0;ot[o][e]&&(i.min=t[o][e]),i.maxt;t++){var f=(t-u)/(p-u),m=240*f,g=this._hsv2rgb(m,1,1);c.strokeStyle=g,c.beginPath(),c.moveTo(h,r+t),c.lineTo(a,r+t),c.stroke()}c.strokeStyle=this.axisColor,c.strokeRect(h,r,i,s)}if(this.style===o.STYLE.DOTSIZE&&(c.strokeStyle=this.axisColor,c.fillStyle=this.dataColor.fill,c.beginPath(),c.moveTo(h,r),c.lineTo(a,r),c.lineTo(a-i+e,d),c.lineTo(h,d),c.closePath(),c.fill(),c.stroke()),this.style===o.STYLE.DOTCOLOR||this.style===o.STYLE.DOTSIZE){var y=5,b=new v(this.valueMin,this.valueMax,(this.valueMax-this.valueMin)/5,!0);for(b.start(),b.getCurrent()0?this.yMin:this.yMax,n=this._convert3Dto2D(new c(_,r,this.zMin)),Math.cos(2*w)>0?(m.textAlign="center",m.textBaseline="top",n.y+=b):Math.sin(2*w)<0?(m.textAlign="right",m.textBaseline="middle"):(m.textAlign="left",m.textBaseline="middle"),m.fillStyle=this.axisColor,m.fillText(" "+this.xValueLabel(i.getCurrent())+" ",n.x,n.y),i.next()}for(m.lineWidth=1,o=void 0===this.defaultYStep,i=new v(this.yMin,this.yMax,this.yStep,o),i.start(),i.getCurrent()0?this.xMin:this.xMax,n=this._convert3Dto2D(new c(s,i.getCurrent(),this.zMin)),Math.cos(2*w)<0?(m.textAlign="center",m.textBaseline="top",n.y+=b):Math.sin(2*w)>0?(m.textAlign="right",m.textBaseline="middle"):(m.textAlign="left",m.textBaseline="middle"),m.fillStyle=this.axisColor,m.fillText(" "+this.yValueLabel(i.getCurrent())+" ",n.x,n.y),i.next();for(m.lineWidth=1,o=void 0===this.defaultZStep,i=new v(this.zMin,this.zMax,this.zStep,o),i.start(),i.getCurrent()0?this.xMin:this.xMax,r=Math.sin(w)<0?this.yMin:this.yMax;!i.end();)t=this._convert3Dto2D(new c(s,r,i.getCurrent())),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(t.x,t.y),m.lineTo(t.x-b,t.y),m.stroke(),m.textAlign="right",m.textBaseline="middle",m.fillStyle=this.axisColor,m.fillText(this.zValueLabel(i.getCurrent())+" ",t.x-5,t.y),i.next();m.lineWidth=1,t=this._convert3Dto2D(new c(s,r,this.zMin)),e=this._convert3Dto2D(new c(s,r,this.zMax)),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(t.x,t.y),m.lineTo(e.x,e.y),m.stroke(),m.lineWidth=1,u=this._convert3Dto2D(new c(this.xMin,this.yMin,this.zMin)),p=this._convert3Dto2D(new c(this.xMax,this.yMin,this.zMin)),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(u.x,u.y),m.lineTo(p.x,p.y),m.stroke(),u=this._convert3Dto2D(new c(this.xMin,this.yMax,this.zMin)),p=this._convert3Dto2D(new c(this.xMax,this.yMax,this.zMin)),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(u.x,u.y),m.lineTo(p.x,p.y),m.stroke(),m.lineWidth=1,t=this._convert3Dto2D(new c(this.xMin,this.yMin,this.zMin)),e=this._convert3Dto2D(new c(this.xMin,this.yMax,this.zMin)),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(t.x,t.y),m.lineTo(e.x,e.y),m.stroke(),t=this._convert3Dto2D(new c(this.xMax,this.yMin,this.zMin)),e=this._convert3Dto2D(new c(this.xMax,this.yMax,this.zMin)),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(t.x,t.y),m.lineTo(e.x,e.y),m.stroke();var x=this.xLabel;x.length>0&&(l=.1/this.scale.y,s=(this.xMin+this.xMax)/2,r=Math.cos(w)>0?this.yMin-l:this.yMax+l,n=this._convert3Dto2D(new c(s,r,this.zMin)),Math.cos(2*w)>0?(m.textAlign="center",m.textBaseline="top"):Math.sin(2*w)<0?(m.textAlign="right",m.textBaseline="middle"):(m.textAlign="left",m.textBaseline="middle"),m.fillStyle=this.axisColor,m.fillText(x,n.x,n.y));var k=this.yLabel;k.length>0&&(d=.1/this.scale.x,s=Math.sin(w)>0?this.xMin-d:this.xMax+d,r=(this.yMin+this.yMax)/2,n=this._convert3Dto2D(new c(s,r,this.zMin)),Math.cos(2*w)<0?(m.textAlign="center",m.textBaseline="top"):Math.sin(2*w)>0?(m.textAlign="right",m.textBaseline="middle"):(m.textAlign="left",m.textBaseline="middle"),m.fillStyle=this.axisColor,m.fillText(k,n.x,n.y));var O=this.zLabel;O.length>0&&(h=30,s=Math.cos(w)>0?this.xMin:this.xMax,r=Math.sin(w)<0?this.yMin:this.yMax,a=(this.zMin+this.zMax)/2,n=this._convert3Dto2D(new c(s,r,a)),m.textAlign="right",m.textBaseline="middle",m.fillStyle=this.axisColor,m.fillText(O,n.x-h,n.y))},o.prototype._hsv2rgb=function(t,e,i){var o,n,s,r,a,h;switch(r=i*e,a=Math.floor(t/60),h=r*(1-Math.abs(t/60%2-1)),a){case 0:o=r,n=h,s=0;break;case 1:o=h,n=r,s=0;break;case 2:o=0,n=r,s=h;break;case 3:o=0,n=h,s=r;break;case 4:o=h,n=0,s=r;break;case 5:o=r,n=0,s=h;break;default:o=0,n=0,s=0}return"RGB("+parseInt(255*o)+","+parseInt(255*n)+","+parseInt(255*s)+")"},o.prototype._redrawDataGrid=function(){var t,e,i,n,s,r,a,h,d,l,u,p,f=this.frame.canvas,m=f.getContext("2d");if(m.lineJoin="round",m.lineCap="round",!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(s=0;s0}else r=!0;r?(p=(t.point.z+e.point.z+i.point.z+n.point.z)/4,d=240*(1-(p-this.zMin)*this.scale.z/this.verticalRatio),l=1,this.showShadow?(u=Math.min(1+x.x/k/2,1),a=this._hsv2rgb(d,l,u),h=a):(u=1,a=this._hsv2rgb(d,l,u),h=this.axisColor)):(a="gray",h=this.axisColor),m.lineWidth=this._getStrokeWidth(t),m.fillStyle=a,m.strokeStyle=h,m.beginPath(),m.moveTo(t.screen.x,t.screen.y),m.lineTo(e.screen.x,e.screen.y),m.lineTo(n.screen.x,n.screen.y),m.lineTo(i.screen.x,i.screen.y),m.closePath(),m.fill(),m.stroke()}}else for(s=0;su&&(u=0);var p,f,m;this.style===o.STYLE.DOTCOLOR?(p=240*(1-(d.point.value-this.valueMin)*this.scale.value),f=this._hsv2rgb(p,1,1),m=this._hsv2rgb(p,1,.8)):this.style===o.STYLE.DOTSIZE?(f=this.dataColor.fill,m=this.dataColor.stroke):(p=240*(1-(d.point.z-this.zMin)*this.scale.z/this.verticalRatio),f=this._hsv2rgb(p,1,1),m=this._hsv2rgb(p,1,.8)),i.lineWidth=this._getStrokeWidth(d),i.strokeStyle=m,i.fillStyle=f,i.beginPath(),i.arc(d.screen.x,d.screen.y,u,0,2*Math.PI,!0),i.fill(),i.stroke()}}},o.prototype._redrawDataBar=function(){var t,e,i,n,s=this.frame.canvas,r=s.getContext("2d");if(!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(t=0;t0){for(t=this.dataPoints[0],o.lineWidth=this._getStrokeWidth(t),o.lineJoin="round",o.lineCap="round",o.strokeStyle=this.dataColor.stroke,o.beginPath(),o.moveTo(t.screen.x,t.screen.y),e=1;e0?1:0>t?-1:0}var o=e[0],n=e[1],s=e[2],r=i((n.x-o.x)*(t.y-o.y)-(n.y-o.y)*(t.x-o.x)),a=i((s.x-n.x)*(t.y-n.y)-(s.y-n.y)*(t.x-n.x)),h=i((o.x-s.x)*(t.y-s.y)-(o.y-s.y)*(t.x-s.x));return!(0!=r&&0!=a&&r!=a||0!=a&&0!=h&&a!=h||0!=r&&0!=h&&r!=h)},o.prototype._dataPointFromXY=function(t,e){var i,n=100,s=null,r=null,a=null,h=new u(t,e);if(this.style===o.STYLE.BAR||this.style===o.STYLE.BARCOLOR||this.style===o.STYLE.BARSIZE)for(i=this.dataPoints.length-1;i>=0;i--){s=this.dataPoints[i];var d=s.surfaces;if(d)for(var l=d.length-1;l>=0;l--){var c=d[l],p=c.corners,f=[p[0].screen,p[1].screen,p[2].screen],m=[p[2].screen,p[3].screen,p[0].screen];if(this._insideTriangle(h,f)||this._insideTriangle(h,m))return s}}else for(i=0;ib)&&n>b&&(a=b,r=s)}}return r},o.prototype._showTooltip=function(t){var e,i,o;this.tooltip?(e=this.tooltip.dom.content,i=this.tooltip.dom.line,o=this.tooltip.dom.dot):(e=document.createElement("div"),e.style.position="absolute",e.style.padding="10px",e.style.border="1px solid #4d4d4d",e.style.color="#1a1a1a",e.style.background="rgba(255,255,255,0.7)",e.style.borderRadius="2px",e.style.boxShadow="5px 5px 10px rgba(128,128,128,0.5)",i=document.createElement("div"),i.style.position="absolute",i.style.height="40px",i.style.width="0",i.style.borderLeft="1px solid #4d4d4d",o=document.createElement("div"),o.style.position="absolute",o.style.height="0",o.style.width="0",o.style.border="5px solid #4d4d4d",o.style.borderRadius="5px",this.tooltip={dataPoint:null,dom:{content:e,line:i,dot:o}}),this._hideTooltip(),this.tooltip.dataPoint=t,"function"==typeof this.showTooltip?e.innerHTML=this.showTooltip(t.point):e.innerHTML="
"+this.xLabel+":"+t.point.x+"
"+this.yLabel+":"+t.point.y+"
"+this.zLabel+":"+t.point.z+"
",e.style.left="0",e.style.top="0",this.frame.appendChild(e),this.frame.appendChild(i),this.frame.appendChild(o);var n=e.offsetWidth,s=e.offsetHeight,r=i.offsetHeight,a=o.offsetWidth,h=o.offsetHeight,d=t.screen.x-n/2;d=Math.min(Math.max(d,10),this.frame.clientWidth-10-n),i.style.left=t.screen.x+"px",i.style.top=t.screen.y-r+"px",e.style.left=d+"px",e.style.top=t.screen.y-r-s+"px",o.style.left=t.screen.x-a/2+"px",o.style.top=t.screen.y-h/2+"px"},o.prototype._hideTooltip=function(){if(this.tooltip){this.tooltip.dataPoint=null;for(var t in this.tooltip.dom)if(this.tooltip.dom.hasOwnProperty(t)){var e=this.tooltip.dom[t];e&&e.parentNode&&e.parentNode.removeChild(e)}}},t.exports=o},function(t,e){function i(t){return t?o(t):void 0}function o(t){for(var e in i.prototype)t[e]=i.prototype[e];return t}t.exports=i,i.prototype.on=i.prototype.addEventListener=function(t,e){return this._callbacks=this._callbacks||{},(this._callbacks[t]=this._callbacks[t]||[]).push(e),this},i.prototype.once=function(t,e){function i(){o.off(t,i),e.apply(this,arguments)}var o=this;return this._callbacks=this._callbacks||{},i.fn=e,this.on(t,i),this},i.prototype.off=i.prototype.removeListener=i.prototype.removeAllListeners=i.prototype.removeEventListener=function(t,e){if(this._callbacks=this._callbacks||{},0==arguments.length)return this._callbacks={},this;var i=this._callbacks[t];if(!i)return this;if(1==arguments.length)return delete this._callbacks[t],this;for(var o,n=0;no;++o)i[o].apply(this,e)}return this},i.prototype.listeners=function(t){return this._callbacks=this._callbacks||{},this._callbacks[t]||[]},i.prototype.hasListeners=function(t){return!!this.listeners(t).length}},function(t,e){function i(t,e,i){this.x=void 0!==t?t:0,this.y=void 0!==e?e:0,this.z=void 0!==i?i:0}i.subtract=function(t,e){var o=new i;return o.x=t.x-e.x,o.y=t.y-e.y,o.z=t.z-e.z,o},i.add=function(t,e){var o=new i;return o.x=t.x+e.x,o.y=t.y+e.y,o.z=t.z+e.z,o},i.avg=function(t,e){return new i((t.x+e.x)/2,(t.y+e.y)/2,(t.z+e.z)/2)},i.crossProduct=function(t,e){var o=new i;return o.x=t.y*e.z-t.z*e.y,o.y=t.z*e.x-t.x*e.z,o.z=t.x*e.y-t.y*e.x,o},i.prototype.length=function(){return Math.sqrt(this.x*this.x+this.y*this.y+this.z*this.z)},t.exports=i},function(t,e){function i(t,e){this.x=void 0!==t?t:0,this.y=void 0!==e?e:0}t.exports=i},function(t,e,i){function o(){this.armLocation=new n,this.armRotation={},this.armRotation.horizontal=0,this.armRotation.vertical=0,this.armLength=1.7,this.cameraLocation=new n,this.cameraRotation=new n(.5*Math.PI,0,0),this.calculateCameraOrientation()}var n=i(14);o.prototype.setArmLocation=function(t,e,i){this.armLocation.x=t,this.armLocation.y=e,this.armLocation.z=i,this.calculateCameraOrientation()},o.prototype.setArmRotation=function(t,e){void 0!==t&&(this.armRotation.horizontal=t),void 0!==e&&(this.armRotation.vertical=e,this.armRotation.vertical<0&&(this.armRotation.vertical=0),this.armRotation.vertical>.5*Math.PI&&(this.armRotation.vertical=.5*Math.PI)),void 0===t&&void 0===e||this.calculateCameraOrientation()},o.prototype.getArmRotation=function(){var t={};return t.horizontal=this.armRotation.horizontal,t.vertical=this.armRotation.vertical,t},o.prototype.setArmLength=function(t){void 0!==t&&(this.armLength=t,this.armLength<.71&&(this.armLength=.71),this.armLength>5&&(this.armLength=5),this.calculateCameraOrientation())},o.prototype.getArmLength=function(){return this.armLength},o.prototype.getCameraLocation=function(){return this.cameraLocation},o.prototype.getCameraRotation=function(){return this.cameraRotation},o.prototype.calculateCameraOrientation=function(){this.cameraLocation.x=this.armLocation.x-this.armLength*Math.sin(this.armRotation.horizontal)*Math.cos(this.armRotation.vertical),this.cameraLocation.y=this.armLocation.y-this.armLength*Math.cos(this.armRotation.horizontal)*Math.cos(this.armRotation.vertical),this.cameraLocation.z=this.armLocation.z+this.armLength*Math.sin(this.armRotation.vertical),this.cameraRotation.x=Math.PI/2-this.armRotation.vertical,this.cameraRotation.y=0,this.cameraRotation.z=-this.armRotation.horizontal},t.exports=o},function(t,e,i){function o(t,e,i){this.data=t,this.column=e,this.graph=i,this.index=void 0,this.value=void 0,this.values=i.getDistinctValues(t.get(),this.column),this.values.sort(function(t,e){return t>e?1:e>t?-1:0}),this.values.length>0&&this.selectValue(0),this.dataPoints=[],this.loaded=!1,this.onLoadCallback=void 0,i.animationPreload?(this.loaded=!1,this.loadInBackground()):this.loaded=!0}var n=i(11);o.prototype.isLoaded=function(){return this.loaded},o.prototype.getLoadedProgress=function(){for(var t=this.values.length,e=0;this.dataPoints[e];)e++;return Math.round(e/t*100)},o.prototype.getLabel=function(){return this.graph.filterLabel},o.prototype.getColumn=function(){return this.column},o.prototype.getSelectedValue=function(){return void 0!==this.index?this.values[this.index]:void 0},o.prototype.getValues=function(){return this.values},o.prototype.getValue=function(t){if(t>=this.values.length)throw"Error: index out of range";return this.values[t]},o.prototype._getDataPoints=function(t){if(void 0===t&&(t=this.index),void 0===t)return[];var e;if(this.dataPoints[t])e=this.dataPoints[t];else{var i={};i.column=this.column,i.value=this.values[t];var o=new n(this.data,{filter:function(t){return t[i.column]==i.value}}).get();e=this.graph._getDataPoints(o),this.dataPoints[t]=e}return e},o.prototype.setOnLoadCallback=function(t){this.onLoadCallback=t},o.prototype.selectValue=function(t){if(t>=this.values.length)throw"Error: index out of range";this.index=t,this.value=this.values[t]},o.prototype.loadInBackground=function(t){void 0===t&&(t=0);var e=this.graph.frame;if(t0&&(t--,this.setIndex(t))},o.prototype.next=function(){var t=this.getIndex();t0?this.setIndex(0):this.index=void 0},o.prototype.setIndex=function(t){if(!(to&&(o=0),o>this.values.length-1&&(o=this.values.length-1),o},o.prototype.indexToLeft=function(t){var e=parseFloat(this.frame.bar.style.width)-this.frame.slide.clientWidth-10,i=t/(this.values.length-1)*e,o=i+3;return o},o.prototype._onMouseMove=function(t){var e=t.clientX-this.startClientX,i=this.startSlideX+e,o=this.leftToIndex(i);this.setIndex(o),n.preventDefault()},o.prototype._onMouseUp=function(t){this.frame.style.cursor="auto",n.removeEventListener(document,"mousemove",this.onmousemove),n.removeEventListener(document,"mouseup",this.onmouseup),n.preventDefault()},t.exports=o},function(t,e){function i(t,e,i,o){this._start=0,this._end=0,this._step=1,this.prettyStep=!0,this.precision=5,this._current=0,this.setRange(t,e,i,o)}i.prototype.setRange=function(t,e,i,o){this._start=t?t:0,this._end=e?e:0,this.setStep(i,o)},i.prototype.setStep=function(t,e){void 0===t||0>=t||(void 0!==e&&(this.prettyStep=e),this.prettyStep===!0?this._step=i.calculatePrettyStep(t):this._step=t)},i.calculatePrettyStep=function(t){var e=function(t){return Math.log(t)/Math.LN10},i=Math.pow(10,Math.round(e(t))),o=2*Math.pow(10,Math.round(e(t/2))),n=5*Math.pow(10,Math.round(e(t/5))),s=i;return Math.abs(o-t)<=Math.abs(s-t)&&(s=o),Math.abs(n-t)<=Math.abs(s-t)&&(s=n),0>=s&&(s=1),s},i.prototype.getCurrent=function(){return parseFloat(this._current.toPrecision(this.precision))},i.prototype.getStep=function(){return this._step},i.prototype.start=function(){this._current=this._start-this._start%this._step},i.prototype.next=function(){this._current+=this._step},i.prototype.end=function(){return this._current>this._end},t.exports=i},function(t,e,i){if("undefined"!=typeof window){var o=i(21),n=window.Hammer||i(22);t.exports=o(n,{preventDefault:"mouse"})}else t.exports=function(){throw Error("hammer.js is only available in a browser, not in node.js.")}},function(t,e,i){var o,n,s;!function(i){n=[],o=i,s="function"==typeof o?o.apply(e,n):o,!(void 0!==s&&(t.exports=s))}(function(){var t=null;return function e(i,o){function n(t){return t.match(/[^ ]+/g)}function s(e){if("hammer.input"!==e.type){if(e.srcEvent._handled||(e.srcEvent._handled={}),e.srcEvent._handled[e.type])return;e.srcEvent._handled[e.type]=!0}var i=!1;e.stopPropagation=function(){i=!0};var o=e.srcEvent.stopPropagation.bind(e.srcEvent);"function"==typeof o&&(e.srcEvent.stopPropagation=function(){o(),e.stopPropagation()}),e.firstTarget=t;for(var n=t;n&&!i;){var s=n.hammer;if(s)for(var r,a=0;a0?d._handlers[t]=o:(i.off(t,s),delete d._handlers[t]))}),d},d.emit=function(e,o){t=o.target,i.emit(e,o)},d.destroy=function(){var t=i.element.hammer,e=t.indexOf(d);-1!==e&&t.splice(e,1),t.length||delete i.element.hammer,d._handlers={},i.destroy()},d}})},function(t,e,i){var o;!function(n,s,r,a){function h(t,e,i){return setTimeout(p(t,i),e)}function d(t,e,i){return Array.isArray(t)?(l(t,i[e],i),!0):!1}function l(t,e,i){var o;if(t)if(t.forEach)t.forEach(e,i);else if(t.length!==a)for(o=0;o\s*\(/gm,"{anonymous}()@"):"Unknown Stack Trace",s=n.console&&(n.console.warn||n.console.log);return s&&s.call(n.console,o,i),t.apply(this,arguments)}}function u(t,e,i){var o,n=e.prototype;o=t.prototype=Object.create(n),o.constructor=t,o._super=n,i&&ct(o,i)}function p(t,e){return function(){return t.apply(e,arguments)}}function f(t,e){return typeof t==ft?t.apply(e?e[0]||a:a,e):t}function m(t,e){return t===a?e:t}function v(t,e,i){l(w(e),function(e){t.addEventListener(e,i,!1)})}function g(t,e,i){l(w(e),function(e){t.removeEventListener(e,i,!1)})}function y(t,e){for(;t;){if(t==e)return!0;t=t.parentNode}return!1}function b(t,e){return t.indexOf(e)>-1}function w(t){return t.trim().split(/\s+/g)}function _(t,e,i){if(t.indexOf&&!i)return t.indexOf(e);for(var o=0;oi[e]}):o.sort()),o}function O(t,e){for(var i,o,n=e[0].toUpperCase()+e.slice(1),s=0;s1&&!i.firstMultiple?i.firstMultiple=N(e):1===n&&(i.firstMultiple=!1);var s=i.firstInput,r=i.firstMultiple,a=r?r.center:s.center,h=e.center=R(o);e.timeStamp=gt(),e.deltaTime=e.timeStamp-s.timeStamp,e.angle=B(a,h),e.distance=A(a,h),P(i,e),e.offsetDirection=L(e.deltaX,e.deltaY);var d=z(e.deltaTime,e.deltaX,e.deltaY);e.overallVelocityX=d.x,e.overallVelocityY=d.y,e.overallVelocity=vt(d.x)>vt(d.y)?d.x:d.y,e.scale=r?j(r.pointers,o):1,e.rotation=r?F(r.pointers,o):0,e.maxPointers=i.prevInput?e.pointers.length>i.prevInput.maxPointers?e.pointers.length:i.prevInput.maxPointers:e.pointers.length,I(i,e);var l=t.element;y(e.srcEvent.target,l)&&(l=e.srcEvent.target),e.target=l}function P(t,e){var i=e.center,o=t.offsetDelta||{},n=t.prevDelta||{},s=t.prevInput||{};e.eventType!==Et&&s.eventType!==It||(n=t.prevDelta={x:s.deltaX||0,y:s.deltaY||0},o=t.offsetDelta={x:i.x,y:i.y}),e.deltaX=n.x+(i.x-o.x),e.deltaY=n.y+(i.y-o.y)}function I(t,e){var i,o,n,s,r=t.lastInterval||e,h=e.timeStamp-r.timeStamp;if(e.eventType!=Nt&&(h>Tt||r.velocity===a)){var d=e.deltaX-r.deltaX,l=e.deltaY-r.deltaY,c=z(h,d,l);o=c.x,n=c.y,i=vt(c.x)>vt(c.y)?c.x:c.y,s=L(d,l),t.lastInterval=e}else i=r.velocity,o=r.velocityX,n=r.velocityY,s=r.direction;e.velocity=i,e.velocityX=o,e.velocityY=n,e.direction=s}function N(t){for(var e=[],i=0;in;)i+=t[n].clientX,o+=t[n].clientY,n++;return{x:mt(i/e),y:mt(o/e)}}function z(t,e,i){return{x:e/t||0,y:i/t||0}}function L(t,e){return t===e?Rt:vt(t)>=vt(e)?0>t?zt:Lt:0>e?At:Bt}function A(t,e,i){i||(i=Wt);var o=e[i[0]]-t[i[0]],n=e[i[1]]-t[i[1]];return Math.sqrt(o*o+n*n)}function B(t,e,i){i||(i=Wt);var o=e[i[0]]-t[i[0]],n=e[i[1]]-t[i[1]];return 180*Math.atan2(n,o)/Math.PI}function F(t,e){return B(e[1],e[0],Yt)+B(t[1],t[0],Yt)}function j(t,e){return A(e[0],e[1],Yt)/A(t[0],t[1],Yt)}function H(){this.evEl=Vt,this.evWin=Ut,this.allow=!0,this.pressed=!1,S.apply(this,arguments)}function W(){this.evEl=Zt,this.evWin=Kt,S.apply(this,arguments),this.store=this.manager.session.pointerEvents=[]}function Y(){this.evTarget=Qt,this.evWin=$t,this.started=!1,S.apply(this,arguments)}function G(t,e){var i=x(t.touches),o=x(t.changedTouches);return e&(It|Nt)&&(i=k(i.concat(o),"identifier",!0)),[i,o]}function V(){this.evTarget=ee,this.targetIds={},S.apply(this,arguments)}function U(t,e){var i=x(t.touches),o=this.targetIds;if(e&(Et|Pt)&&1===i.length)return o[i[0].identifier]=!0,[i,i];var n,s,r=x(t.changedTouches),a=[],h=this.target;if(s=i.filter(function(t){return y(t.target,h)}),e===Et)for(n=0;na&&(e.push(t),a=e.length-1):n&(It|Nt)&&(i=!0),0>a||(e[a]=t,this.callback(this.manager,n,{pointers:e,changedPointers:[t],pointerType:s,srcEvent:t}),i&&e.splice(a,1))}});var Jt={touchstart:Et,touchmove:Pt,touchend:It,touchcancel:Nt},Qt="touchstart",$t="touchstart touchmove touchend touchcancel";u(Y,S,{handler:function(t){var e=Jt[t.type];if(e===Et&&(this.started=!0),this.started){var i=G.call(this,t,e);e&(It|Nt)&&i[0].length-i[1].length===0&&(this.started=!1),this.callback(this.manager,e,{pointers:i[0],changedPointers:i[1],pointerType:Mt,srcEvent:t})}}});var te={touchstart:Et,touchmove:Pt,touchend:It,touchcancel:Nt},ee="touchstart touchmove touchend touchcancel";u(V,S,{handler:function(t){var e=te[t.type],i=U.call(this,t,e);i&&this.callback(this.manager,e,{pointers:i[0],changedPointers:i[1],pointerType:Mt,srcEvent:t})}}),u(q,S,{handler:function(t,e,i){var o=i.pointerType==Mt,n=i.pointerType==St;if(o)this.mouse.allow=!1;else if(n&&!this.mouse.allow)return;e&(It|Nt)&&(this.mouse.allow=!0),this.callback(t,e,i)},destroy:function(){this.touch.destroy(),this.mouse.destroy()}});var ie=O(pt.style,"touchAction"),oe=ie!==a,ne="compute",se="auto",re="manipulation",ae="none",he="pan-x",de="pan-y";X.prototype={set:function(t){t==ne&&(t=this.compute()),oe&&this.manager.element.style&&(this.manager.element.style[ie]=t),this.actions=t.toLowerCase().trim()},update:function(){this.set(this.manager.options.touchAction)},compute:function(){var t=[];return l(this.manager.recognizers,function(e){f(e.options.enable,[e])&&(t=t.concat(e.getTouchAction()))}),Z(t.join(" "))},preventDefaults:function(t){if(!oe){var e=t.srcEvent,i=t.offsetDirection;if(this.manager.session.prevented)return void e.preventDefault();var o=this.actions,n=b(o,ae),s=b(o,de),r=b(o,he);if(n){var a=1===t.pointers.length,h=t.distance<2,d=t.deltaTime<250;if(a&&h&&d)return}if(!r||!s)return n||s&&i&Ft||r&&i&jt?this.preventSrc(e):void 0}},preventSrc:function(t){this.manager.session.prevented=!0,t.preventDefault()}};var le=1,ce=2,ue=4,pe=8,fe=pe,me=16,ve=32;K.prototype={defaults:{},set:function(t){return ct(this.options,t),this.manager&&this.manager.touchAction.update(),this},recognizeWith:function(t){if(d(t,"recognizeWith",this))return this;var e=this.simultaneous;return t=$(t,this),e[t.id]||(e[t.id]=t,t.recognizeWith(this)),this},dropRecognizeWith:function(t){return d(t,"dropRecognizeWith",this)?this:(t=$(t,this),delete this.simultaneous[t.id],this)},requireFailure:function(t){if(d(t,"requireFailure",this))return this;var e=this.requireFail;return t=$(t,this),-1===_(e,t)&&(e.push(t),t.requireFailure(this)),this},dropRequireFailure:function(t){if(d(t,"dropRequireFailure",this))return this;t=$(t,this);var e=_(this.requireFail,t);return e>-1&&this.requireFail.splice(e,1),this},hasRequireFailures:function(){return this.requireFail.length>0},canRecognizeWith:function(t){return!!this.simultaneous[t.id]},emit:function(t){function e(e){i.manager.emit(e,t)}var i=this,o=this.state;pe>o&&e(i.options.event+J(o)),e(i.options.event),t.additionalEvent&&e(t.additionalEvent),o>=pe&&e(i.options.event+J(o))},tryEmit:function(t){return this.canEmit()?this.emit(t):void(this.state=ve)},canEmit:function(){for(var t=0;ts?zt:Lt,i=s!=this.pX,o=Math.abs(t.deltaX)):(n=0===r?Rt:0>r?At:Bt,i=r!=this.pY,o=Math.abs(t.deltaY))),t.direction=n,i&&o>e.threshold&&n&e.direction},attrTest:function(t){return tt.prototype.attrTest.call(this,t)&&(this.state&ce||!(this.state&ce)&&this.directionTest(t))},emit:function(t){this.pX=t.deltaX,this.pY=t.deltaY;var e=Q(t.direction);e&&(t.additionalEvent=this.options.event+e),this._super.emit.call(this,t)}}),u(it,tt,{defaults:{event:"pinch",threshold:0,pointers:2},getTouchAction:function(){return[ae]},attrTest:function(t){return this._super.attrTest.call(this,t)&&(Math.abs(t.scale-1)>this.options.threshold||this.state&ce)},emit:function(t){if(1!==t.scale){var e=t.scale<1?"in":"out";t.additionalEvent=this.options.event+e}this._super.emit.call(this,t)}}),u(ot,K,{defaults:{event:"press",pointers:1,time:251,threshold:9},getTouchAction:function(){return[se]},process:function(t){var e=this.options,i=t.pointers.length===e.pointers,o=t.distancee.time;if(this._input=t,!o||!i||t.eventType&(It|Nt)&&!n)this.reset();else if(t.eventType&Et)this.reset(),this._timer=h(function(){this.state=fe,this.tryEmit()},e.time,this);else if(t.eventType&It)return fe;return ve},reset:function(){clearTimeout(this._timer)},emit:function(t){this.state===fe&&(t&&t.eventType&It?this.manager.emit(this.options.event+"up",t):(this._input.timeStamp=gt(),this.manager.emit(this.options.event,this._input)))}}),u(nt,tt,{defaults:{event:"rotate",threshold:0,pointers:2},getTouchAction:function(){return[ae]},attrTest:function(t){return this._super.attrTest.call(this,t)&&(Math.abs(t.rotation)>this.options.threshold||this.state&ce)}}),u(st,tt,{defaults:{event:"swipe",threshold:10,velocity:.3,direction:Ft|jt,pointers:1},getTouchAction:function(){return et.prototype.getTouchAction.call(this)},attrTest:function(t){var e,i=this.options.direction;return i&(Ft|jt)?e=t.overallVelocity:i&Ft?e=t.overallVelocityX:i&jt&&(e=t.overallVelocityY),this._super.attrTest.call(this,t)&&i&t.offsetDirection&&t.distance>this.options.threshold&&t.maxPointers==this.options.pointers&&vt(e)>this.options.velocity&&t.eventType&It},emit:function(t){var e=Q(t.offsetDirection);e&&this.manager.emit(this.options.event+e,t),this.manager.emit(this.options.event,t)}}),u(rt,K,{defaults:{event:"tap",pointers:1,taps:1,interval:300,time:250,threshold:9,posThreshold:10},getTouchAction:function(){return[re]},process:function(t){var e=this.options,i=t.pointers.length===e.pointers,o=t.distance=e;e++)r[String.fromCharCode(e)]={code:65+(e-97),shift:!1};for(e=65;90>=e;e++)r[String.fromCharCode(e)]={code:e,shift:!0};for(e=0;9>=e;e++)r[""+e]={code:48+e,shift:!1};for(e=1;12>=e;e++)r["F"+e]={code:111+e,shift:!1};for(e=0;9>=e;e++)r["num"+e]={code:96+e,shift:!1};r["num*"]={code:106,shift:!1},r["num+"]={code:107,shift:!1},r["num-"]={code:109,shift:!1},r["num/"]={code:111,shift:!1},r["num."]={code:110,shift:!1},r.left={code:37,shift:!1},r.up={code:38,shift:!1},r.right={code:39,shift:!1},r.down={code:40,shift:!1},r.space={code:32,shift:!1},r.enter={code:13,shift:!1},r.shift={code:16,shift:void 0},r.esc={code:27,shift:!1},r.backspace={code:8,shift:!1},r.tab={code:9,shift:!1},r.ctrl={code:17,shift:!1},r.alt={code:18,shift:!1},r["delete"]={code:46,shift:!1},r.pageup={code:33,shift:!1},r.pagedown={code:34,shift:!1},r["="]={code:187,shift:!1},r["-"]={code:189,shift:!1},r["]"]={code:221,shift:!1},r["["]={code:219,shift:!1};var a=function(t){d(t,"keydown")},h=function(t){d(t,"keyup")},d=function(t,e){if(void 0!==s[e][t.keyCode]){for(var o=s[e][t.keyCode],n=0;ne)&&(n=e),(null===s||i>s)&&(s=i)}),null!==n&&null!==s){var r=(n+s)/2,a=Math.max(this.range.end-this.range.start,1.1*(s-n)),h=e&&void 0!==e.animation?e.animation:!0;this.range.setRange(r-a/2,r+a/2,h)}}},n.prototype.fit=function(t){var e,i=t&&void 0!==t.animation?t.animation:!0,o=this.itemsData&&this.itemsData.getDataSet();1===o.length&&void 0===o.get()[0].end?(e=this.getDataRange(),this.moveTo(e.min.valueOf(),{animation:i})):(e=this.getItemRange(),this.range.setRange(e.min,e.max,i))},n.prototype.getItemRange=function(){var t=this,e=this.getDataRange(),i=null!==e.min?e.min.valueOf():null,o=null!==e.max?e.max.valueOf():null,n=null,s=null;if(null!=i&&null!=o){var r,a,h,d,c;!function(){var e=function(t){return l.convert(t.data.start,"Date").valueOf()},u=function(t){var e=void 0!=t.data.end?t.data.end:t.data.start;return l.convert(e,"Date").valueOf()};r=o-i,0>=r&&(r=10),a=r/t.props.center.width,l.forEach(t.itemSet.items,function(t){t.show(),t.repositionX();var r=e(t),h=u(t);if(this.options.rtl)var d=r-(t.getWidthRight()+10)*a,l=h+(t.getWidthLeft()+10)*a;else var d=r-(t.getWidthLeft()+10)*a,l=h+(t.getWidthRight()+10)*a;i>d&&(i=d,n=t),l>o&&(o=l,s=t)}.bind(t)),n&&s&&(h=n.getWidthLeft()+10,d=s.getWidthRight()+10,c=t.props.center.width-h-d,c>0&&(t.options.rtl?(i=e(n)-d*r/c,o=u(s)+h*r/c):(i=e(n)-h*r/c,o=u(s)+d*r/c)))}()}return{min:null!=i?new Date(i):null,max:null!=o?new Date(o):null}},n.prototype.getDataRange=function(){var t=null,e=null,i=this.itemsData&&this.itemsData.getDataSet();return i&&i.forEach(function(i){var o=l.convert(i.start,"Date").valueOf(),n=l.convert(void 0!=i.end?i.end:i.start,"Date").valueOf();(null===t||t>o)&&(t=o),(null===e||n>e)&&(e=n)}),{min:null!=t?new Date(t):null,max:null!=e?new Date(e):null}},n.prototype.getEventProperties=function(t){var e=t.center?t.center.x:t.clientX,i=t.center?t.center.y:t.clientY;if(this.options.rtl)var o=l.getAbsoluteRight(this.dom.centerContainer)-e;else var o=e-l.getAbsoluteLeft(this.dom.centerContainer);var n=i-l.getAbsoluteTop(this.dom.centerContainer),s=this.itemSet.itemFromTarget(t),r=this.itemSet.groupFromTarget(t),a=g.customTimeFromTarget(t),h=this.itemSet.options.snap||null,d=this.body.util.getScale(),c=this.body.util.getStep(),u=this._toTime(o),p=h?h(u,d,c):u,f=l.getTarget(t),m=null;return null!=s?m="item":null!=a?m="custom-time":l.hasParent(f,this.timeAxis.dom.foreground)?m="axis":this.timeAxis2&&l.hasParent(f,this.timeAxis2.dom.foreground)?m="axis":l.hasParent(f,this.itemSet.dom.labelSet)?m="group-label":l.hasParent(f,this.currentTime.bar)?m="current-time":l.hasParent(f,this.dom.center)&&(m="background"),{event:t,item:s?s.id:null,group:r?r.groupId:null,what:m,pageX:t.srcEvent?t.srcEvent.pageX:t.pageX,pageY:t.srcEvent?t.srcEvent.pageY:t.pageY,x:o,y:n,time:u,snappedTime:p}},t.exports=n},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},r=function(){function t(t,e){for(var i=0;i0&&this._makeItem([]),this._makeHeader(n),this._handleObject(this.configureOptions[n],[n])),i++);this.options.showButton===!0&&!function(){var e=document.createElement("div");e.className="vis-configuration vis-config-button",e.innerHTML="generate options",e.onclick=function(){t._printOptions()},e.onmouseover=function(){e.className="vis-configuration vis-config-button hover"},e.onmouseout=function(){e.className="vis-configuration vis-config-button"},t.optionsContainer=document.createElement("div"),t.optionsContainer.className="vis-configuration vis-config-option-container",t.domElements.push(t.optionsContainer),t.domElements.push(e)}(),this._push()}},{key:"_push",value:function(){this.wrapper=document.createElement("div"),this.wrapper.className="vis-configuration-wrapper",this.container.appendChild(this.wrapper);for(var t=0;t1?o-1:0),r=1;o>r;r++)n[r-1]=e[r];return n.forEach(function(t){s.appendChild(t)}),i.domElements.push(s),{v:i.domElements.length}}();if("object"===("undefined"==typeof a?"undefined":s(a)))return a.v}return 0}},{key:"_makeHeader",value:function(t){var e=document.createElement("div");e.className="vis-configuration vis-config-header",e.innerHTML=t,this._makeItem([],e)}},{key:"_makeLabel",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!1:arguments[2],o=document.createElement("div");return o.className="vis-configuration vis-config-label vis-config-s"+e.length,i===!0?o.innerHTML=""+t+":":o.innerHTML=t+":",o}},{key:"_makeDropdown",value:function(t,e,i){var o=document.createElement("select");o.className="vis-configuration vis-config-select";var n=0;void 0!==e&&-1!==t.indexOf(e)&&(n=t.indexOf(e));for(var s=0;se&&n>e*c?(a.min=Math.ceil(e*c),l=a.min,d="range increased"):n>e/c&&(a.min=Math.ceil(e/c),l=a.min,d="range increased"),e*c>s&&1!==s&&(a.max=Math.ceil(e*c),l=a.max,d="range increased"),a.value=e}else a.value=o;var u=document.createElement("input");u.className="vis-configuration vis-config-rangeinput",u.value=a.value;var p=this;a.onchange=function(){u.value=this.value,p._update(Number(this.value),i)},a.oninput=function(){u.value=this.value};var f=this._makeLabel(i[i.length-1],i),m=this._makeItem(i,f,a,u);""!==d&&this.popupHistory[m]!==l&&(this.popupHistory[m]=l,this._setupPopup(d,m))}},{key:"_setupPopup",value:function(t,e){var i=this;if(this.initialized===!0&&this.allowCreation===!0&&this.popupCountervar options = "+JSON.stringify(t,null,2)+""}},{key:"getOptions",value:function(){for(var t={},e=0;es;s++)for(r=0;rp?p+1:p;var f=l/this.r,m=a.RGBToHSV(this.color.r,this.color.g,this.color.b);m.h=p,m.s=f;var v=a.HSVToRGB(m.h,m.s,m.v);v.a=this.color.a,this.color=v,this.initialColorDiv.style.backgroundColor="rgba("+this.initialColor.r+","+this.initialColor.g+","+this.initialColor.b+","+this.initialColor.a+")",this.newColorDiv.style.backgroundColor="rgba("+this.color.r+","+this.color.g+","+this.color.b+","+this.color.a+")"}}]),t}();e["default"]=h},function(t,e,i){i(20);e.onTouch=function(t,e){e.inputHandler=function(t){t.isFirst&&e(t)},t.on("hammer.input",e.inputHandler)},e.onRelease=function(t,e){return e.inputHandler=function(t){t.isFinal&&e(t)},t.on("hammer.input",e.inputHandler)},e.offTouch=function(t,e){t.off("hammer.input",e.inputHandler)},e.offRelease=e.offTouch,e.disablePreventDefaultVertically=function(t){var e="pan-y";return t.getTouchAction=function(){return[e]},t}},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=function(){function t(t,e){for(var i=0;is.distance?console.log('%cUnknown option detected: "'+e+'" in '+t.printLocation(n.path,e,"")+"Perhaps it was misplaced? Matching option found at: "+t.printLocation(s.path,s.closestMatch,""),d):n.distance<=r?console.log('%cUnknown option detected: "'+e+'". Did you mean "'+n.closestMatch+'"?'+t.printLocation(n.path,e),d):console.log('%cUnknown option detected: "'+e+'". Did you mean one of these: '+t.print(Object.keys(i))+t.printLocation(o,e),d),a=!0}},{key:"findInOptions",value:function(e,i,o){var n=arguments.length<=3||void 0===arguments[3]?!1:arguments[3],s=1e9,a="",h=[],d=e.toLowerCase(),l=void 0;for(var c in i){var u=void 0;if(void 0!==i[c].__type__&&n===!0){var p=t.findInOptions(e,i[c],r.copyAndExtendArray(o,c));s>p.distance&&(a=p.closestMatch,h=p.path,s=p.distance,l=p.indexMatch)}else-1!==c.toLowerCase().indexOf(d)&&(l=c),u=t.levenshteinDistance(e,c),s>u&&(a=c,h=r.copyArray(o),s=u)}return{closestMatch:a,path:h,distance:s,indexMatch:l}}},{key:"printLocation",value:function(t,e){for(var i=arguments.length<=2||void 0===arguments[2]?"Problem value found at: \n":arguments[2],o="\n\n"+i+"options = {\n",n=0;ns;s++)o+=" ";o+=t[n]+": {\n"}for(var r=0;ru,r=s||null===n?n:l+(n-l)*i,p=s||null===a?a:c+(a-c)*i;y=h._applyRange(r,p),d.updateHiddenDates(h.options.moment,h.body,h.options.hiddenDates),v=v||y,y&&h.body.emitter.emit("rangechange",{start:new Date(h.start),end:new Date(h.end),byUser:o}),s?v&&h.body.emitter.emit("rangechanged",{start:new Date(h.start),end:new Date(h.end),byUser:o}):h.animationTimer=setTimeout(w,20)}};return g()}var y=this._applyRange(n,a);if(d.updateHiddenDates(this.options.moment,this.body,this.options.hiddenDates),y){var b={start:new Date(this.start),end:new Date(this.end),byUser:o};this.body.emitter.emit("rangechange",b),this.body.emitter.emit("rangechanged",b)}},o.prototype._cancelAnimation=function(){this.animationTimer&&(clearTimeout(this.animationTimer),this.animationTimer=null)},o.prototype._applyRange=function(t,e){var i,o=null!=t?r.convert(t,"Date").valueOf():this.start,n=null!=e?r.convert(e,"Date").valueOf():this.end,s=null!=this.options.max?r.convert(this.options.max,"Date").valueOf():null,a=null!=this.options.min?r.convert(this.options.min,"Date").valueOf():null;if(isNaN(o)||null===o)throw new Error('Invalid start "'+t+'"');if(isNaN(n)||null===n)throw new Error('Invalid end "'+e+'"');if(o>n&&(n=o),null!==a&&a>o&&(i=a-o,o+=i,n+=i,null!=s&&n>s&&(n=s)),null!==s&&n>s&&(i=n-s,o-=i,n-=i,null!=a&&a>o&&(o=a)),null!==this.options.zoomMin){var h=parseFloat(this.options.zoomMin);0>h&&(h=0),h>n-o&&(this.end-this.start===h&&o>this.start&&nd&&(d=0),n-o>d&&(this.end-this.start===d&&othis.end?(o=this.start,n=this.end):(i=n-o-d,o+=i/2,n-=i/2))}var l=this.start!=o||this.end!=n;return o>=this.start&&o<=this.end||n>=this.start&&n<=this.end||this.start>=o&&this.start<=n||this.end>=o&&this.end<=n||this.body.emitter.emit("checkRangedItems"),this.start=o,this.end=n,l},o.prototype.getRange=function(){return{start:this.start,end:this.end}},o.prototype.conversion=function(t,e){return o.conversion(this.start,this.end,t,e)},o.conversion=function(t,e,i,o){return void 0===o&&(o=0),0!=i&&e-t!=0?{offset:t,scale:i/(e-t-o)}:{offset:0,scale:1}},o.prototype._onDragStart=function(t){this.deltaDifference=0,this.previousDelta=0,this.options.moveable&&this._isInsideRange(t)&&this.props.touch.allowDragging&&(this.props.touch.start=this.start,this.props.touch.end=this.end,this.props.touch.dragging=!0,this.body.dom.root&&(this.body.dom.root.style.cursor="move"))},o.prototype._onDrag=function(t){if(this.props.touch.dragging&&this.options.moveable&&this.props.touch.allowDragging){var e=this.options.direction;n(e);var i="horizontal"==e?t.deltaX:t.deltaY;i-=this.deltaDifference;var o=this.props.touch.end-this.props.touch.start,s=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end);o-=s;var r="horizontal"==e?this.body.domProps.center.width:this.body.domProps.center.height;if(this.options.rtl)var a=i/r*o;else var a=-i/r*o;var h=this.props.touch.start+a,l=this.props.touch.end+a,c=d.snapAwayFromHidden(this.body.hiddenDates,h,this.previousDelta-i,!0),u=d.snapAwayFromHidden(this.body.hiddenDates,l,this.previousDelta-i,!0);if(c!=h||u!=l)return this.deltaDifference+=i,this.props.touch.start=c,this.props.touch.end=u,void this._onDrag(t);this.previousDelta=i,this._applyRange(h,l);var p=new Date(this.start),f=new Date(this.end);this.body.emitter.emit("rangechange",{start:p,end:f,byUser:!0})}},o.prototype._onDragEnd=function(t){this.props.touch.dragging&&this.options.moveable&&this.props.touch.allowDragging&&(this.props.touch.dragging=!1,this.body.dom.root&&(this.body.dom.root.style.cursor="auto"),this.body.emitter.emit("rangechanged",{start:new Date(this.start),end:new Date(this.end),byUser:!0}))},o.prototype._onMouseWheel=function(t){if(this.options.zoomable&&this.options.moveable&&this._isInsideRange(t)&&(!this.options.zoomKey||t[this.options.zoomKey])){var e=0;if(t.wheelDelta?e=t.wheelDelta/120:t.detail&&(e=-t.detail/3),e){var i;i=0>e?1-e/5:1/(1+e/5);var o=this.getPointer({x:t.clientX,y:t.clientY},this.body.dom.center),n=this._pointerToDate(o);this.zoom(i,n,e)}t.preventDefault()}},o.prototype._onTouch=function(t){this.props.touch.start=this.start,this.props.touch.end=this.end,this.props.touch.allowDragging=!0,this.props.touch.center=null,this.scaleOffset=0,this.deltaDifference=0},o.prototype._onPinch=function(t){if(this.options.zoomable&&this.options.moveable){this.props.touch.allowDragging=!1,this.props.touch.center||(this.props.touch.center=this.getPointer(t.center,this.body.dom.center));var e=1/(t.scale+this.scaleOffset),i=this._pointerToDate(this.props.touch.center),o=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end),n=d.getHiddenDurationBefore(this.options.moment,this.body.hiddenDates,this,i),s=o-n,r=i-n+(this.props.touch.start-(i-n))*e,a=i+s+(this.props.touch.end-(i+s))*e; +this.startToFront=0>=1-e,this.endToFront=0>=e-1;var h=d.snapAwayFromHidden(this.body.hiddenDates,r,1-e,!0),l=d.snapAwayFromHidden(this.body.hiddenDates,a,e-1,!0);h==r&&l==a||(this.props.touch.start=h,this.props.touch.end=l,this.scaleOffset=1-t.scale,r=h,a=l),this.setRange(r,a,!1,!0),this.startToFront=!1,this.endToFront=!0}},o.prototype._isInsideRange=function(t){var e=t.center?t.center.x:t.clientX;if(this.options.rtl)var i=e-r.getAbsoluteLeft(this.body.dom.centerContainer);else var i=r.getAbsoluteRight(this.body.dom.centerContainer)-e;var o=this.body.util.toTime(i);return o>=this.start&&o<=this.end},o.prototype._pointerToDate=function(t){var e,i=this.options.direction;if(n(i),"horizontal"==i)return this.body.util.toTime(t.x).valueOf();var o=this.body.domProps.center.height;return e=this.conversion(o),t.y/e.scale+e.offset},o.prototype.getPointer=function(t,e){return this.options.rtl?{x:r.getAbsoluteRight(e)-t.x,y:t.y-r.getAbsoluteTop(e)}:{x:t.x-r.getAbsoluteLeft(e),y:t.y-r.getAbsoluteTop(e)}},o.prototype.zoom=function(t,e,i){null==e&&(e=(this.start+this.end)/2);var o=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end),n=d.getHiddenDurationBefore(this.options.moment,this.body.hiddenDates,this,e),s=o-n,r=e-n+(this.start-(e-n))*t,a=e+s+(this.end-(e+s))*t;this.startToFront=!(i>0),this.endToFront=!(-i>0);var h=d.snapAwayFromHidden(this.body.hiddenDates,r,i,!0),l=d.snapAwayFromHidden(this.body.hiddenDates,a,-i,!0);h==r&&l==a||(r=h,a=l),this.setRange(r,a,!1,!0),this.startToFront=!1,this.endToFront=!0},o.prototype.move=function(t){var e=this.end-this.start,i=this.start+e*t,o=this.end+e*t;this.start=i,this.end=o},o.prototype.moveTo=function(t){var e=(this.start+this.end)/2,i=e-t,o=this.start-i,n=this.end-i;this.setRange(o,n)},t.exports=o},function(t,e){function i(t,e){this.options=null,this.props=null}i.prototype.setOptions=function(t){t&&util.extend(this.options,t)},i.prototype.redraw=function(){return!1},i.prototype.destroy=function(){},i.prototype._isResized=function(){var t=this.props._previousWidth!==this.props.width||this.props._previousHeight!==this.props.height;return this.props._previousWidth=this.props.width,this.props._previousHeight=this.props.height,t},t.exports=i},function(t,e){e.convertHiddenOptions=function(t,i,o){if(o&&!Array.isArray(o))return e.convertHiddenOptions(t,i,[o]);if(i.hiddenDates=[],o&&1==Array.isArray(o)){for(var n=0;n=4*a){var u=0,p=s.clone();switch(o[h].repeat){case"daily":d.day()!=l.day()&&(u=1),d.dayOfYear(n.dayOfYear()),d.year(n.year()),d.subtract(7,"days"),l.dayOfYear(n.dayOfYear()),l.year(n.year()),l.subtract(7-u,"days"),p.add(1,"weeks");break;case"weekly":var f=l.diff(d,"days"),m=d.day();d.date(n.date()),d.month(n.month()),d.year(n.year()),l=d.clone(),d.day(m),l.day(m),l.add(f,"days"),d.subtract(1,"weeks"),l.subtract(1,"weeks"),p.add(1,"weeks");break;case"monthly":d.month()!=l.month()&&(u=1),d.month(n.month()),d.year(n.year()),d.subtract(1,"months"),l.month(n.month()),l.year(n.year()),l.subtract(1,"months"),l.add(u,"months"),p.add(1,"months");break;case"yearly":d.year()!=l.year()&&(u=1),d.year(n.year()),d.subtract(1,"years"),l.year(n.year()),l.subtract(1,"years"),l.add(u,"years"),p.add(1,"years");break;default:return void console.log("Wrong repeat format, allowed are: daily, weekly, monthly, yearly. Given:",o[h].repeat)}for(;p>d;)switch(i.hiddenDates.push({start:d.valueOf(),end:l.valueOf()}),o[h].repeat){case"daily":d.add(1,"days"),l.add(1,"days");break;case"weekly":d.add(1,"weeks"),l.add(1,"weeks");break;case"monthly":d.add(1,"months"),l.add(1,"months");break;case"yearly":d.add(1,"y"),l.add(1,"y");break;default:return void console.log("Wrong repeat format, allowed are: daily, weekly, monthly, yearly. Given:",o[h].repeat)}i.hiddenDates.push({start:d.valueOf(),end:l.valueOf()})}}e.removeDuplicates(i);var v=e.isHidden(i.range.start,i.hiddenDates),g=e.isHidden(i.range.end,i.hiddenDates),y=i.range.start,b=i.range.end;1==v.hidden&&(y=1==i.range.startToFront?v.startDate-1:v.endDate+1),1==g.hidden&&(b=1==i.range.endToFront?g.startDate-1:g.endDate+1),1!=v.hidden&&1!=g.hidden||i.range._applyRange(y,b)}},e.removeDuplicates=function(t){for(var e=t.hiddenDates,i=[],o=0;o=e[o].start&&e[n].end<=e[o].end?e[n].remove=!0:e[n].start>=e[o].start&&e[n].start<=e[o].end?(e[o].end=e[n].end,e[n].remove=!0):e[n].end>=e[o].start&&e[n].end<=e[o].end&&(e[o].start=e[n].start,e[n].remove=!0));for(var o=0;o=r&&a>n){o=!0;break}}if(1==o&&n=e&&i>r&&(o+=r-s)}return o},e.correctTimeForHidden=function(t,i,o,n){return n=t(n).toDate().valueOf(),n-=e.getHiddenDurationBefore(t,i,o,n)},e.getHiddenDurationBefore=function(t,e,i,o){var n=0;o=t(o).toDate().valueOf();for(var s=0;s=i.start&&a=a&&(n+=a-r)}return n},e.getAccumulatedHiddenDuration=function(t,e,i){for(var o=0,n=0,s=e.start,r=0;r=e.start&&h=i)break;o+=h-a}}return o},e.snapAwayFromHidden=function(t,i,o,n){var s=e.isHidden(i,t);return 1==s.hidden?0>o?1==n?s.startDate-(s.endDate-i)-1:s.startDate-1:1==n?s.endDate+(i-s.startDate)+1:s.endDate+1:i},e.isHidden=function(t,e){for(var i=0;i=o&&n>t)return{hidden:!0,startDate:o,endDate:n}}return{hidden:!1,startDate:o,endDate:n}}},function(t,e,i){function o(){}var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=i(13),r=i(20),a=i(28),h=i(1),d=(i(9),i(11),i(30),i(34),i(44)),l=i(45),c=i(32),u=i(46);s(o.prototype),o.prototype._create=function(t){function e(t){i.isActive()&&i.emit("mousewheel",t)}this.dom={},this.dom.container=t,this.dom.root=document.createElement("div"),this.dom.background=document.createElement("div"),this.dom.backgroundVertical=document.createElement("div"),this.dom.backgroundHorizontal=document.createElement("div"),this.dom.centerContainer=document.createElement("div"),this.dom.leftContainer=document.createElement("div"),this.dom.rightContainer=document.createElement("div"),this.dom.center=document.createElement("div"),this.dom.left=document.createElement("div"),this.dom.right=document.createElement("div"),this.dom.top=document.createElement("div"),this.dom.bottom=document.createElement("div"),this.dom.shadowTop=document.createElement("div"),this.dom.shadowBottom=document.createElement("div"),this.dom.shadowTopLeft=document.createElement("div"),this.dom.shadowBottomLeft=document.createElement("div"),this.dom.shadowTopRight=document.createElement("div"),this.dom.shadowBottomRight=document.createElement("div"),this.dom.root.className="vis-timeline",this.dom.background.className="vis-panel vis-background",this.dom.backgroundVertical.className="vis-panel vis-background vis-vertical",this.dom.backgroundHorizontal.className="vis-panel vis-background vis-horizontal",this.dom.centerContainer.className="vis-panel vis-center",this.dom.leftContainer.className="vis-panel vis-left",this.dom.rightContainer.className="vis-panel vis-right",this.dom.top.className="vis-panel vis-top",this.dom.bottom.className="vis-panel vis-bottom",this.dom.left.className="vis-content",this.dom.center.className="vis-content",this.dom.right.className="vis-content",this.dom.shadowTop.className="vis-shadow vis-top",this.dom.shadowBottom.className="vis-shadow vis-bottom",this.dom.shadowTopLeft.className="vis-shadow vis-top",this.dom.shadowBottomLeft.className="vis-shadow vis-bottom",this.dom.shadowTopRight.className="vis-shadow vis-top",this.dom.shadowBottomRight.className="vis-shadow vis-bottom",this.dom.root.appendChild(this.dom.background),this.dom.root.appendChild(this.dom.backgroundVertical),this.dom.root.appendChild(this.dom.backgroundHorizontal),this.dom.root.appendChild(this.dom.centerContainer),this.dom.root.appendChild(this.dom.leftContainer),this.dom.root.appendChild(this.dom.rightContainer),this.dom.root.appendChild(this.dom.top),this.dom.root.appendChild(this.dom.bottom),this.dom.centerContainer.appendChild(this.dom.center),this.dom.leftContainer.appendChild(this.dom.left),this.dom.rightContainer.appendChild(this.dom.right),this.dom.centerContainer.appendChild(this.dom.shadowTop),this.dom.centerContainer.appendChild(this.dom.shadowBottom),this.dom.leftContainer.appendChild(this.dom.shadowTopLeft),this.dom.leftContainer.appendChild(this.dom.shadowBottomLeft),this.dom.rightContainer.appendChild(this.dom.shadowTopRight),this.dom.rightContainer.appendChild(this.dom.shadowBottomRight),this.on("rangechange",function(){this.initialDrawDone===!0&&this._redraw()}.bind(this)),this.on("touch",this._onTouch.bind(this)),this.on("pan",this._onDrag.bind(this));var i=this;this.on("_change",function(t){t&&1==t.queue?i._redrawTimer||(i._redrawTimer=setTimeout(function(){i._redrawTimer=null,i._redraw()},0)):i._redraw()}),this.hammer=new r(this.dom.root);var o=this.hammer.get("pinch").set({enable:!0});a.disablePreventDefaultVertically(o),this.hammer.get("pan").set({threshold:5,direction:r.DIRECTION_HORIZONTAL}),this.listeners={};var n=["tap","doubletap","press","pinch","pan","panstart","panmove","panend"];if(n.forEach(function(t){var e=function(e){i.isActive()&&i.emit(t,e)};i.hammer.on(t,e),i.listeners[t]=e}),a.onTouch(this.hammer,function(t){i.emit("touch",t)}.bind(this)),a.onRelease(this.hammer,function(t){i.emit("release",t)}.bind(this)),this.dom.root.addEventListener("mousewheel",e),this.dom.root.addEventListener("DOMMouseScroll",e),this.props={root:{},background:{},centerContainer:{},leftContainer:{},rightContainer:{},center:{},left:{},right:{},top:{},bottom:{},border:{},scrollTop:0,scrollTopMin:0},this.customTimes=[],this.touch={},this.redrawCount=0,this.initialDrawDone=!1,!t)throw new Error("No container provided");t.appendChild(this.dom.root)},o.prototype.setOptions=function(t){if(t){var e=["width","height","minHeight","maxHeight","autoResize","start","end","clickToUse","dataAttributes","hiddenDates","locale","locales","moment","rtl","throttleRedraw"];if(h.selectiveExtend(e,this.options,t),this.options.rtl){var i=this.dom.leftContainer;this.dom.leftContainer=this.dom.rightContainer,this.dom.rightContainer=i,this.dom.container.style.direction="rtl",this.dom.backgroundVertical.className="vis-panel vis-background vis-vertical-rtl"}if(this.options.orientation={item:void 0,axis:void 0},"orientation"in t&&("string"==typeof t.orientation?this.options.orientation={item:t.orientation,axis:t.orientation}:"object"===n(t.orientation)&&("item"in t.orientation&&(this.options.orientation.item=t.orientation.item),"axis"in t.orientation&&(this.options.orientation.axis=t.orientation.axis))),"both"===this.options.orientation.axis){if(!this.timeAxis2){var o=this.timeAxis2=new d(this.body);o.setOptions=function(t){var e=t?h.extend({},t):{};e.orientation="top",d.prototype.setOptions.call(o,e)},this.components.push(o)}}else if(this.timeAxis2){var s=this.components.indexOf(this.timeAxis2);-1!==s&&this.components.splice(s,1),this.timeAxis2.destroy(),this.timeAxis2=null}if("function"==typeof t.drawPoints&&(t.drawPoints={onRender:t.drawPoints}),"hiddenDates"in this.options&&c.convertHiddenOptions(this.options.moment,this.body,this.options.hiddenDates),"clickToUse"in t&&(t.clickToUse?this.activator||(this.activator=new l(this.dom.root)):this.activator&&(this.activator.destroy(),delete this.activator)),"showCustomTime"in t)throw new Error("Option `showCustomTime` is deprecated. Create a custom time bar via timeline.addCustomTime(time [, id])");this._initAutoResize()}if(this.components.forEach(function(e){return e.setOptions(t)}),"configure"in t){this.configurator||(this.configurator=this._createConfigurator()),this.configurator.setOptions(t.configure);var r=h.deepExtend({},this.options);this.components.forEach(function(t){h.deepExtend(r,t.options)}),this.configurator.setModuleOptions({global:r})}this._origRedraw?this._redraw():(this._origRedraw=this._redraw.bind(this),this._redraw=h.throttle(this._origRedraw,this.options.throttleRedraw))},o.prototype.isActive=function(){return!this.activator||this.activator.active},o.prototype.destroy=function(){this.setItems(null),this.setGroups(null),this.off(),this._stopAutoResize(),this.dom.root.parentNode&&this.dom.root.parentNode.removeChild(this.dom.root),this.dom=null,this.activator&&(this.activator.destroy(),delete this.activator);for(var t in this.listeners)this.listeners.hasOwnProperty(t)&&delete this.listeners[t];this.listeners=null,this.hammer=null,this.components.forEach(function(t){return t.destroy()}),this.body=null},o.prototype.setCustomTime=function(t,e){var i=this.customTimes.filter(function(t){return e===t.options.id});if(0===i.length)throw new Error("No custom time bar found with id "+JSON.stringify(e));i.length>0&&i[0].setCustomTime(t)},o.prototype.getCustomTime=function(t){var e=this.customTimes.filter(function(e){return e.options.id===t});if(0===e.length)throw new Error("No custom time bar found with id "+JSON.stringify(t));return e[0].getCustomTime()},o.prototype.setCustomTimeTitle=function(t,e){var i=this.customTimes.filter(function(t){return t.options.id===e});if(0===i.length)throw new Error("No custom time bar found with id "+JSON.stringify(e));return i.length>0?i[0].setCustomTitle(t):void 0},o.prototype.getEventProperties=function(t){return{event:t}},o.prototype.addCustomTime=function(t,e){var i=void 0!==t?h.convert(t,"Date").valueOf():new Date,o=this.customTimes.some(function(t){return t.options.id===e});if(o)throw new Error("A custom time with id "+JSON.stringify(e)+" already exists");var n=new u(this.body,h.extend({},this.options,{time:i,id:e}));return this.customTimes.push(n),this.components.push(n),this._redraw(),e},o.prototype.removeCustomTime=function(t){var e=this.customTimes.filter(function(e){return e.options.id===t});if(0===e.length)throw new Error("No custom time bar found with id "+JSON.stringify(t));e.forEach(function(t){this.customTimes.splice(this.customTimes.indexOf(t),1),this.components.splice(this.components.indexOf(t),1),t.destroy()}.bind(this))},o.prototype.getVisibleItems=function(){return this.itemSet&&this.itemSet.getVisibleItems()||[]},o.prototype.fit=function(t){var e=this.getDataRange();if(null!==e.min||null!==e.max){var i=e.max-e.min,o=new Date(e.min.valueOf()-.01*i),n=new Date(e.max.valueOf()+.01*i),s=t&&void 0!==t.animation?t.animation:!0;this.range.setRange(o,n,s)}},o.prototype.getDataRange=function(){throw new Error("Cannot invoke abstract method getDataRange")},o.prototype.setWindow=function(t,e,i){var o;if(1==arguments.length){var n=arguments[0];o=void 0!==n.animation?n.animation:!0,this.range.setRange(n.start,n.end,o)}else o=i&&void 0!==i.animation?i.animation:!0,this.range.setRange(t,e,o)},o.prototype.moveTo=function(t,e){var i=this.range.end-this.range.start,o=h.convert(t,"Date").valueOf(),n=o-i/2,s=o+i/2,r=e&&void 0!==e.animation?e.animation:!0;this.range.setRange(n,s,r)},o.prototype.getWindow=function(){var t=this.range.getRange();return{start:new Date(t.start),end:new Date(t.end)}},o.prototype.redraw=function(){this._redraw()},o.prototype._redraw=function(){this.redrawCount++;var t=!1,e=this.options,i=this.props,o=this.dom;if(o&&o.container&&0!=o.root.offsetWidth){c.updateHiddenDates(this.options.moment,this.body,this.options.hiddenDates),"top"==e.orientation?(h.addClassName(o.root,"vis-top"),h.removeClassName(o.root,"vis-bottom")):(h.removeClassName(o.root,"vis-top"),h.addClassName(o.root,"vis-bottom")),o.root.style.maxHeight=h.option.asSize(e.maxHeight,""),o.root.style.minHeight=h.option.asSize(e.minHeight,""),o.root.style.width=h.option.asSize(e.width,""),i.border.left=(o.centerContainer.offsetWidth-o.centerContainer.clientWidth)/2,i.border.right=i.border.left,i.border.top=(o.centerContainer.offsetHeight-o.centerContainer.clientHeight)/2,i.border.bottom=i.border.top;var n=o.root.offsetHeight-o.root.clientHeight,s=o.root.offsetWidth-o.root.clientWidth;0===o.centerContainer.clientHeight&&(i.border.left=i.border.top,i.border.right=i.border.left),0===o.root.clientHeight&&(s=n),i.center.height=o.center.offsetHeight,i.left.height=o.left.offsetHeight,i.right.height=o.right.offsetHeight,i.top.height=o.top.clientHeight||-i.border.top,i.bottom.height=o.bottom.clientHeight||-i.border.bottom;var a=Math.max(i.left.height,i.center.height,i.right.height),d=i.top.height+a+i.bottom.height+n+i.border.top+i.border.bottom;o.root.style.height=h.option.asSize(e.height,d+"px"),i.root.height=o.root.offsetHeight,i.background.height=i.root.height-n;var l=i.root.height-i.top.height-i.bottom.height-n;i.centerContainer.height=l,i.leftContainer.height=l,i.rightContainer.height=i.leftContainer.height,i.root.width=o.root.offsetWidth,i.background.width=i.root.width-s,i.left.width=o.leftContainer.clientWidth||-i.border.left,i.leftContainer.width=i.left.width,i.right.width=o.rightContainer.clientWidth||-i.border.right,i.rightContainer.width=i.right.width;var u=i.root.width-i.left.width-i.right.width-s;i.center.width=u,i.centerContainer.width=u,i.top.width=u,i.bottom.width=u,o.background.style.height=i.background.height+"px",o.backgroundVertical.style.height=i.background.height+"px",o.backgroundHorizontal.style.height=i.centerContainer.height+"px",o.centerContainer.style.height=i.centerContainer.height+"px",o.leftContainer.style.height=i.leftContainer.height+"px",o.rightContainer.style.height=i.rightContainer.height+"px",o.background.style.width=i.background.width+"px",o.backgroundVertical.style.width=i.centerContainer.width+"px",o.backgroundHorizontal.style.width=i.background.width+"px",o.centerContainer.style.width=i.center.width+"px",o.top.style.width=i.top.width+"px",o.bottom.style.width=i.bottom.width+"px",o.background.style.left="0",o.background.style.top="0",o.backgroundVertical.style.left=i.left.width+i.border.left+"px",o.backgroundVertical.style.top="0",o.backgroundHorizontal.style.left="0",o.backgroundHorizontal.style.top=i.top.height+"px",o.centerContainer.style.left=i.left.width+"px",o.centerContainer.style.top=i.top.height+"px",o.leftContainer.style.left="0",o.leftContainer.style.top=i.top.height+"px",o.rightContainer.style.left=i.left.width+i.center.width+"px",o.rightContainer.style.top=i.top.height+"px",o.top.style.left=i.left.width+"px",o.top.style.top="0",o.bottom.style.left=i.left.width+"px",o.bottom.style.top=i.top.height+i.centerContainer.height+"px",this._updateScrollTop();var p=this.props.scrollTop;"top"!=e.orientation.item&&(p+=Math.max(this.props.centerContainer.height-this.props.center.height-this.props.border.top-this.props.border.bottom,0)),o.center.style.left="0",o.center.style.top=p+"px",o.left.style.left="0",o.left.style.top=p+"px",o.right.style.left="0",o.right.style.top=p+"px";var f=0==this.props.scrollTop?"hidden":"",m=this.props.scrollTop==this.props.scrollTopMin?"hidden":"";o.shadowTop.style.visibility=f,o.shadowBottom.style.visibility=m,o.shadowTopLeft.style.visibility=f,o.shadowBottomLeft.style.visibility=m,o.shadowTopRight.style.visibility=f,o.shadowBottomRight.style.visibility=m;var v=this.props.center.height>this.props.centerContainer.height;this.hammer.get("pan").set({direction:v?r.DIRECTION_ALL:r.DIRECTION_HORIZONTAL}),this.components.forEach(function(e){t=e.redraw()||t});var g=5;if(t){if(this.redrawCount0&&(this.props.scrollTop=0),this.props.scrollTope;e++)o=this.selection[e],n=this.items[o],n&&n.unselect();for(this.selection=[],e=0,i=t.length;i>e;e++)o=t[e],n=this.items[o],n&&(this.selection.push(o),n.select())},o.prototype.getSelection=function(){return this.selection.concat([])},o.prototype.getVisibleItems=function(){var t=this.body.range.getRange();if(this.options.rtl)var e=this.body.util.toScreen(t.start),i=this.body.util.toScreen(t.end);else var i=this.body.util.toScreen(t.start),e=this.body.util.toScreen(t.end);var o=[];for(var n in this.groups)if(this.groups.hasOwnProperty(n))for(var s=this.groups[n],r=s.visibleItems,a=0;ae&&o.push(h.id):h.lefti&&o.push(h.id)}return o},o.prototype._deselect=function(t){for(var e=this.selection,i=0,o=e.length;o>i;i++)if(e[i]==t){e.splice(i,1);break}},o.prototype.redraw=function(){var t=this.options.margin,e=this.body.range,i=r.option.asSize,o=this.options,n=o.orientation.item,s=!1,a=this.dom.frame;this.props.top=this.body.domProps.top.height+this.body.domProps.border.top,this.options.rtl?this.props.right=this.body.domProps.right.width+this.body.domProps.border.right:this.props.left=this.body.domProps.left.width+this.body.domProps.border.left,a.className="vis-itemset",s=this._orderGroups()||s;var h=e.end-e.start,d=h!=this.lastVisibleInterval||this.props.width!=this.props.lastWidth;d&&(this.stackDirty=!0), +this.lastVisibleInterval=h,this.props.lastWidth=this.props.width;var l=this.stackDirty,c=this._firstGroup(),u={item:t.item,axis:t.axis},p={item:t.item,axis:t.item.vertical/2},f=0,m=t.axis+t.item.vertical;return this.groups[y].redraw(e,p,l),r.forEach(this.groups,function(t){var i=t==c?u:p,o=t.redraw(e,i,l);s=o||s,f+=t.height}),f=Math.max(f,m),this.stackDirty=!1,a.style.height=i(f),this.props.width=a.offsetWidth,this.props.height=f,this.dom.axis.style.top=i("top"==n?this.body.domProps.top.height+this.body.domProps.border.top:this.body.domProps.top.height+this.body.domProps.centerContainer.height),this.options.rtl?this.dom.axis.style.right="0":this.dom.axis.style.left="0",s=this._isResized()||s},o.prototype._firstGroup=function(){var t="top"==this.options.orientation.item?0:this.groupIds.length-1,e=this.groupIds[t],i=this.groups[e]||this.groups[g];return i||null},o.prototype._updateUngrouped=function(){var t,e,i=this.groups[g];this.groups[y];if(this.groupsData){if(i){i.hide(),delete this.groups[g];for(e in this.items)if(this.items.hasOwnProperty(e)){t=this.items[e],t.parent&&t.parent.remove(t);var o=this._getGroupId(t.data),n=this.groups[o];n&&n.add(t)||t.hide()}}}else if(!i){var s=null,r=null;i=new c(s,r,this),this.groups[g]=i;for(e in this.items)this.items.hasOwnProperty(e)&&(t=this.items[e],i.add(t));i.show()}},o.prototype.getLabelSet=function(){return this.dom.labelSet},o.prototype.setItems=function(t){var e,i=this,o=this.itemsData;if(t){if(!(t instanceof a||t instanceof h))throw new TypeError("Data must be an instance of DataSet or DataView");this.itemsData=t}else this.itemsData=null;if(o&&(r.forEach(this.itemListeners,function(t,e){o.off(e,t)}),e=o.getIds(),this._onRemove(e)),this.itemsData){var n=this.id;r.forEach(this.itemListeners,function(t,e){i.itemsData.on(e,t,n)}),e=this.itemsData.getIds(),this._onAdd(e),this._updateUngrouped()}this.body.emitter.emit("_change",{queue:!0})},o.prototype.getItems=function(){return this.itemsData},o.prototype.setGroups=function(t){var e,i=this;if(this.groupsData&&(r.forEach(this.groupListeners,function(t,e){i.groupsData.off(e,t)}),e=this.groupsData.getIds(),this.groupsData=null,this._onRemoveGroups(e)),t){if(!(t instanceof a||t instanceof h))throw new TypeError("Data must be an instance of DataSet or DataView");this.groupsData=t}else this.groupsData=null;if(this.groupsData){var o=this.id;r.forEach(this.groupListeners,function(t,e){i.groupsData.on(e,t,o)}),e=this.groupsData.getIds(),this._onAddGroups(e)}this._updateUngrouped(),this._order(),this.body.emitter.emit("_change",{queue:!0})},o.prototype.getGroups=function(){return this.groupsData},o.prototype.removeItem=function(t){var e=this.itemsData.get(t),i=this.itemsData.getDataSet();e&&this.options.onRemove(e,function(e){e&&i.remove(t)})},o.prototype._getType=function(t){return t.type||this.options.type||(t.end?"range":"box")},o.prototype._getGroupId=function(t){var e=this._getType(t);return"background"==e&&void 0==t.group?y:this.groupsData?t.group:g},o.prototype._onUpdate=function(t){var e=this;t.forEach(function(t){var i,n=e.itemsData.get(t,e.itemOptions),s=e.items[t],r=e._getType(n),a=o.types[r];if(s&&(a&&s instanceof a?e._updateItem(s,n):(i=s.selected,e._removeItem(s),s=null)),!s){if(!a)throw"rangeoverflow"==r?new TypeError('Item type "rangeoverflow" is deprecated. Use css styling instead: .vis-item.vis-range .vis-item-content {overflow: visible;}'):new TypeError('Unknown item type "'+r+'"');s=new a(n,e.conversion,e.options),s.id=t,e._addItem(s),i&&(this.selection.push(t),s.select())}}.bind(this)),this._order(),this.stackDirty=!0,this.body.emitter.emit("_change",{queue:!0})},o.prototype._onAdd=o.prototype._onUpdate,o.prototype._onRemove=function(t){var e=0,i=this;t.forEach(function(t){var o=i.items[t];o&&(e++,i._removeItem(o))}),e&&(this._order(),this.stackDirty=!0,this.body.emitter.emit("_change",{queue:!0}))},o.prototype._order=function(){r.forEach(this.groups,function(t){t.order()})},o.prototype._onUpdateGroups=function(t){this._onAddGroups(t)},o.prototype._onAddGroups=function(t){var e=this;t.forEach(function(t){var i=e.groupsData.get(t),o=e.groups[t];if(o)o.setData(i);else{if(t==g||t==y)throw new Error("Illegal group id. "+t+" is a reserved id.");var n=Object.create(e.options);r.extend(n,{height:null}),o=new c(t,i,e),e.groups[t]=o;for(var s in e.items)if(e.items.hasOwnProperty(s)){var a=e.items[s];a.data.group==t&&o.add(a)}o.order(),o.show()}}),this.body.emitter.emit("_change",{queue:!0})},o.prototype._onRemoveGroups=function(t){var e=this.groups;t.forEach(function(t){var i=e[t];i&&(i.hide(),delete e[t])}),this.markDirty(),this.body.emitter.emit("_change",{queue:!0})},o.prototype._orderGroups=function(){if(this.groupsData){var t=this.groupsData.getIds({order:this.options.groupOrder}),e=!r.equalArray(t,this.groupIds);if(e){var i=this.groups;t.forEach(function(t){i[t].hide()}),t.forEach(function(t){i[t].show()}),this.groupIds=t}return e}return!1},o.prototype._addItem=function(t){this.items[t.id]=t;var e=this._getGroupId(t.data),i=this.groups[e];i&&i.add(t)},o.prototype._updateItem=function(t,e){var i=t.data.group,o=t.data.subgroup;if(t.setData(e),i!=t.data.group||o!=t.data.subgroup){var n=this.groups[i];n&&n.remove(t);var s=this._getGroupId(t.data),r=this.groups[s];r&&r.add(t)}},o.prototype._removeItem=function(t){t.hide(),delete this.items[t.id];var e=this.selection.indexOf(t.id);-1!=e&&this.selection.splice(e,1),t.parent&&t.parent.remove(t)},o.prototype._constructByEndArray=function(t){for(var e=[],i=0;in+s)return}else{var a=e.height;if(n+a-s>o)return}}if(e&&e!=this.groupTouchParams.group){var h=this.groupsData,d=h.get(e.groupId),l=h.get(this.groupTouchParams.group.groupId);l&&d&&(this.options.groupOrderSwap(l,d,this.groupsData),this.groupsData.update(l),this.groupsData.update(d));var c=this.groupsData.getIds({order:this.options.groupOrder});if(!r.equalArray(c,this.groupTouchParams.originalOrder))for(var h=this.groupsData,u=this.groupTouchParams.originalOrder,p=this.groupTouchParams.group.groupId,f=Math.min(u.length,c.length),m=0,v=0,g=0;f>m;){for(;f>m+v&&f>m+g&&c[m+v]==u[m+g];)m++;if(m+v>=f)break;if(c[m+v]!=p)if(u[m+g]!=p){var y=c.indexOf(u[m+g]),b=h.get(c[m+v]),w=h.get(u[m+g]);this.options.groupOrderSwap(b,w,h),h.update(b),h.update(w);var _=c[m+v];c[m+v]=u[m+g],c[y]=_,m++}else g=1;else v=1}}}},o.prototype._onGroupDragEnd=function(t){if(this.options.groupEditable.order&&this.groupTouchParams.group){t.stopPropagation();var e=this,i=e.groupTouchParams.group.groupId,o=e.groupsData.getDataSet(),n=r.extend({},o.get(i));e.options.onMoveGroup(n,function(t){if(t)t[o._fieldId]=i,o.update(t);else{var n=o.getIds({order:e.options.groupOrder});if(!r.equalArray(n,e.groupTouchParams.originalOrder))for(var s=e.groupTouchParams.originalOrder,a=Math.min(s.length,n.length),h=0;a>h;){for(;a>h&&n[h]==s[h];)h++;if(h>=a)break;var d=n.indexOf(s[h]),l=o.get(n[h]),c=o.get(s[h]);e.options.groupOrderSwap(l,c,o),groupsData.update(l),groupsData.update(c);var u=n[h];n[h]=s[h],n[d]=u,h++}}}),e.body.emitter.emit("groupDragged",{groupId:i})}},o.prototype._onSelectItem=function(t){if(this.options.selectable){var e=t.srcEvent&&(t.srcEvent.ctrlKey||t.srcEvent.metaKey),i=t.srcEvent&&t.srcEvent.shiftKey;if(e||i)return void this._onMultiSelectItem(t);var o=this.getSelection(),n=this.itemFromTarget(t),s=n?[n.id]:[];this.setSelection(s);var r=this.getSelection();(r.length>0||o.length>0)&&this.body.emitter.emit("select",{items:r,event:t})}},o.prototype._onAddItem=function(t){if(this.options.selectable&&this.options.editable.add){var e=this,i=this.options.snap||null,o=this.itemFromTarget(t);if(o){var n=e.itemsData.get(o.id);this.options.onUpdate(n,function(t){t&&e.itemsData.getDataSet().update(t)})}else{if(this.options.rtl)var s=r.getAbsoluteRight(this.dom.frame),a=s-t.center.x;else var s=r.getAbsoluteLeft(this.dom.frame),a=t.center.x-s;var h=this.body.util.toTime(a),d=this.body.util.getScale(),l=this.body.util.getStep(),c={start:i?i(h,d,l):h,content:"new item"};if("range"===this.options.type){var u=this.body.util.toTime(a+this.props.width/5);c.end=i?i(u,d,l):u}c[this.itemsData._fieldId]=r.randomUUID();var p=this.groupFromTarget(t);p&&(c.group=p.groupId),c=this._cloneItemData(c),this.options.onAdd(c,function(t){t&&e.itemsData.getDataSet().add(t)})}}},o.prototype._onMultiSelectItem=function(t){if(this.options.selectable){var e=this.itemFromTarget(t);if(e){var i=this.options.multiselect?this.getSelection():[],n=t.srcEvent&&t.srcEvent.shiftKey||!1;if(n&&this.options.multiselect){var s=this.itemsData.get(e.id).group,r=void 0;this.options.multiselectPerGroup&&i.length>0&&(r=this.itemsData.get(i[0]).group),this.options.multiselectPerGroup&&void 0!=r&&r!=s||i.push(e.id);var a=o._getItemRange(this.itemsData.get(i,this.itemOptions));if(!this.options.multiselectPerGroup||r==s){i=[];for(var h in this.items)if(this.items.hasOwnProperty(h)){var d=this.items[h],l=d.data.start,c=void 0!==d.data.end?d.data.end:l;!(l>=a.min&&c<=a.max)||this.options.multiselectPerGroup&&r!=this.itemsData.get(d.id).group||d instanceof v||i.push(d.id)}}}else{var u=i.indexOf(e.id);-1==u?i.push(e.id):i.splice(u,1)}this.setSelection(i),this.body.emitter.emit("select",{items:this.getSelection(),event:t})}}},o._getItemRange=function(t){var e=null,i=null;return t.forEach(function(t){(null==i||t.starte)&&(e=t.end):(null==e||t.start>e)&&(e=t.start)}),{min:i,max:e}},o.prototype.itemFromTarget=function(t){for(var e=t.target;e;){if(e.hasOwnProperty("timeline-item"))return e["timeline-item"];e=e.parentNode}return null},o.prototype.groupFromTarget=function(t){for(var e=t.center?t.center.y:t.clientY,i=0;ia&&ea)return n}else if(0===i&&e0?t.step:1,this.autoScale=!1)},o.prototype.setAutoScale=function(t){this.autoScale=t},o.prototype.setMinimumStep=function(t){if(void 0!=t){var e=31104e6,i=2592e6,o=864e5,n=36e5,s=6e4,r=1e3,a=1;1e3*e>t&&(this.scale="year",this.step=1e3),500*e>t&&(this.scale="year",this.step=500),100*e>t&&(this.scale="year",this.step=100),50*e>t&&(this.scale="year",this.step=50),10*e>t&&(this.scale="year",this.step=10),5*e>t&&(this.scale="year",this.step=5),e>t&&(this.scale="year",this.step=1),3*i>t&&(this.scale="month",this.step=3),i>t&&(this.scale="month",this.step=1),5*o>t&&(this.scale="day",this.step=5),2*o>t&&(this.scale="day",this.step=2),o>t&&(this.scale="day",this.step=1),o/2>t&&(this.scale="weekday",this.step=1),4*n>t&&(this.scale="hour",this.step=4),n>t&&(this.scale="hour",this.step=1),15*s>t&&(this.scale="minute",this.step=15),10*s>t&&(this.scale="minute",this.step=10),5*s>t&&(this.scale="minute",this.step=5),s>t&&(this.scale="minute",this.step=1),15*r>t&&(this.scale="second",this.step=15),10*r>t&&(this.scale="second",this.step=10),5*r>t&&(this.scale="second",this.step=5),r>t&&(this.scale="second",this.step=1),200*a>t&&(this.scale="millisecond",this.step=200),100*a>t&&(this.scale="millisecond",this.step=100),50*a>t&&(this.scale="millisecond",this.step=50),10*a>t&&(this.scale="millisecond",this.step=10),5*a>t&&(this.scale="millisecond",this.step=5),a>t&&(this.scale="millisecond",this.step=1)}},o.snap=function(t,e,i){var o=n(t);if("year"==e){var s=o.year()+Math.round(o.month()/12);o.year(Math.round(s/i)*i),o.month(0),o.date(0),o.hours(0),o.minutes(0),o.seconds(0),o.milliseconds(0)}else if("month"==e)o.date()>15?(o.date(1),o.add(1,"month")):o.date(1),o.hours(0),o.minutes(0),o.seconds(0),o.milliseconds(0);else if("day"==e){switch(i){case 5:case 2:o.hours(24*Math.round(o.hours()/24));break;default:o.hours(12*Math.round(o.hours()/12))}o.minutes(0),o.seconds(0),o.milliseconds(0)}else if("weekday"==e){switch(i){case 5:case 2:o.hours(12*Math.round(o.hours()/12));break;default:o.hours(6*Math.round(o.hours()/6))}o.minutes(0),o.seconds(0),o.milliseconds(0)}else if("hour"==e){switch(i){case 4:o.minutes(60*Math.round(o.minutes()/60));break;default:o.minutes(30*Math.round(o.minutes()/30))}o.seconds(0),o.milliseconds(0)}else if("minute"==e){switch(i){case 15:case 10:o.minutes(5*Math.round(o.minutes()/5)),o.seconds(0);break;case 5:o.seconds(60*Math.round(o.seconds()/60));break;default:o.seconds(30*Math.round(o.seconds()/30))}o.milliseconds(0)}else if("second"==e)switch(i){case 15:case 10:o.seconds(5*Math.round(o.seconds()/5)),o.milliseconds(0);break;case 5:o.milliseconds(1e3*Math.round(o.milliseconds()/1e3));break;default:o.milliseconds(500*Math.round(o.milliseconds()/500))}else if("millisecond"==e){var r=i>5?i/2:1;o.milliseconds(Math.round(o.milliseconds()/r)*r)}return o},o.prototype.isMajor=function(){if(1==this.switchedYear)switch(this.switchedYear=!1,this.scale){case"year":case"month":case"weekday":case"day":case"hour":case"minute":case"second":case"millisecond":return!0;default:return!1}else if(1==this.switchedMonth)switch(this.switchedMonth=!1,this.scale){case"weekday":case"day":case"hour":case"minute":case"second":case"millisecond":return!0;default:return!1}else if(1==this.switchedDay)switch(this.switchedDay=!1,this.scale){case"millisecond":case"second":case"minute":case"hour":return!0;default:return!1}var t=this.moment(this.current);switch(this.scale){case"millisecond":return 0==t.milliseconds();case"second":return 0==t.seconds();case"minute":return 0==t.hours()&&0==t.minutes();case"hour":return 0==t.hours();case"weekday":case"day":return 1==t.date();case"month":return 0==t.month();case"year":return!1;default:return!1}},o.prototype.getLabelMinor=function(t){void 0==t&&(t=this.current);var e=this.format.minorLabels[this.scale];return e&&e.length>0?this.moment(t).format(e):""},o.prototype.getLabelMajor=function(t){void 0==t&&(t=this.current);var e=this.format.majorLabels[this.scale];return e&&e.length>0?this.moment(t).format(e):""},o.prototype.getClassName=function(){function t(t){return t/h%2==0?" vis-even":" vis-odd"}function e(t){return t.isSame(new Date,"day")?" vis-today":t.isSame(s().add(1,"day"),"day")?" vis-tomorrow":t.isSame(s().add(-1,"day"),"day")?" vis-yesterday":""}function i(t){return t.isSame(new Date,"week")?" vis-current-week":""}function o(t){return t.isSame(new Date,"month")?" vis-current-month":""}function n(t){return t.isSame(new Date,"year")?" vis-current-year":""}var s=this.moment,r=this.moment(this.current),a=r.locale?r.locale("en"):r.lang("en"),h=this.step;switch(this.scale){case"millisecond":return t(a.milliseconds()).trim();case"second":return t(a.seconds()).trim();case"minute":return t(a.minutes()).trim();case"hour":var d=a.hours();return 4==this.step&&(d=d+"-h"+(d+4)),"vis-h"+d+e(a)+t(a.hours());case"weekday":return"vis-"+a.format("dddd").toLowerCase()+e(a)+i(a)+t(a.date());case"day":var l=a.date(),c=a.format("MMMM").toLowerCase();return"vis-day"+l+" vis-"+c+o(a)+t(l-1);case"month":return"vis-"+a.format("MMMM").toLowerCase()+o(a)+t(a.month());case"year":var u=a.year();return"vis-year"+u+n(a)+t(u);default:return""}},t.exports=o},function(t,e,i){function o(t,e,i){this.groupId=t,this.subgroups={},this.subgroupIndex=0,this.subgroupOrderer=e&&e.subgroupOrder,this.itemSet=i,this.dom={},this.props={label:{width:0,height:0}},this.className=null,this.items={},this.visibleItems=[],this.orderedItems={byStart:[],byEnd:[]},this.checkRangedItems=!1;var o=this;this.itemSet.body.emitter.on("checkRangedItems",function(){o.checkRangedItems=!0}),this._create(),this.setData(e)}var n=i(1),s=i(37);i(38);o.prototype._create=function(){var t=document.createElement("div");this.itemSet.options.groupEditable.order?t.className="vis-label draggable":t.className="vis-label",this.dom.label=t;var e=document.createElement("div");e.className="vis-inner",t.appendChild(e),this.dom.inner=e;var i=document.createElement("div");i.className="vis-group",i["timeline-group"]=this,this.dom.foreground=i,this.dom.background=document.createElement("div"),this.dom.background.className="vis-group",this.dom.axis=document.createElement("div"),this.dom.axis.className="vis-group",this.dom.marker=document.createElement("div"),this.dom.marker.style.visibility="hidden",this.dom.marker.innerHTML="?",this.dom.background.appendChild(this.dom.marker)},o.prototype.setData=function(t){var e;if(e=this.itemSet.options&&this.itemSet.options.groupTemplate?this.itemSet.options.groupTemplate(t):t&&t.content,e instanceof Element){for(this.dom.inner.appendChild(e);this.dom.inner.firstChild;)this.dom.inner.removeChild(this.dom.inner.firstChild);this.dom.inner.appendChild(e)}else void 0!==e&&null!==e?this.dom.inner.innerHTML=e:this.dom.inner.innerHTML=this.groupId||"";this.dom.label.title=t&&t.title||"",this.dom.inner.firstChild?n.removeClassName(this.dom.inner,"vis-hidden"):n.addClassName(this.dom.inner,"vis-hidden");var i=t&&t.className||null;i!=this.className&&(this.className&&(n.removeClassName(this.dom.label,this.className),n.removeClassName(this.dom.foreground,this.className),n.removeClassName(this.dom.background,this.className),n.removeClassName(this.dom.axis,this.className)),n.addClassName(this.dom.label,i),n.addClassName(this.dom.foreground,i),n.addClassName(this.dom.background,i),n.addClassName(this.dom.axis,i),this.className=i),this.style&&(n.removeCssText(this.dom.label,this.style),this.style=null),t&&t.style&&(n.addCssText(this.dom.label,t.style),this.style=t.style)},o.prototype.getLabelWidth=function(){return this.props.label.width},o.prototype.redraw=function(t,e,i){var o=!1,r=this.dom.marker.clientHeight;if(r!=this.lastMarkerHeight&&(this.lastMarkerHeight=r,n.forEach(this.items,function(t){t.dirty=!0,t.displayed&&t.redraw()}),i=!0),this._calculateSubGroupHeights(),"function"==typeof this.itemSet.options.order){if(i){var a=this,h=!1;n.forEach(this.items,function(t){t.displayed||(t.redraw(),a.visibleItems.push(t)),t.repositionX(h)});var d=this.orderedItems.byStart.slice().sort(function(t,e){return a.itemSet.options.order(t.data,e.data)});s.stack(d,e,!0)}this.visibleItems=this._updateVisibleItems(this.orderedItems,this.visibleItems,t)}else this.visibleItems=this._updateVisibleItems(this.orderedItems,this.visibleItems,t),this.itemSet.options.stack?s.stack(this.visibleItems,e,i):s.nostack(this.visibleItems,e,this.subgroups);var l=this._calculateHeight(e),c=this.dom.foreground;this.top=c.offsetTop,this.right=c.offsetLeft,this.width=c.offsetWidth,o=n.updateProperty(this,"height",l)||o,o=n.updateProperty(this.props.label,"width",this.dom.inner.clientWidth)||o,o=n.updateProperty(this.props.label,"height",this.dom.inner.clientHeight)||o,this.dom.background.style.height=l+"px",this.dom.foreground.style.height=l+"px",this.dom.label.style.height=l+"px";for(var u=0,p=this.visibleItems.length;p>u;u++){var f=this.visibleItems[u];f.repositionY(e)}return o},o.prototype._calculateSubGroupHeights=function(){if(Object.keys(this.subgroups).length>0){var t=this;this.resetSubgroups(),n.forEach(this.visibleItems,function(e){void 0!==e.data.subgroup&&(t.subgroups[e.data.subgroup].height=Math.max(t.subgroups[e.data.subgroup].height,e.height),t.subgroups[e.data.subgroup].visible=!0)})}},o.prototype._calculateHeight=function(t){var e,i=this.visibleItems;if(i.length>0){var o=i[0].top,s=i[0].top+i[0].height;if(n.forEach(i,function(t){o=Math.min(o,t.top),s=Math.max(s,t.top+t.height)}),o>t.axis){var r=o-t.axis;s-=r,n.forEach(i,function(t){t.top-=r})}e=s+t.item.vertical/2}else e=0;return e=Math.max(e,this.props.label.height)},o.prototype.show=function(){this.dom.label.parentNode||this.itemSet.dom.labelSet.appendChild(this.dom.label),this.dom.foreground.parentNode||this.itemSet.dom.foreground.appendChild(this.dom.foreground),this.dom.background.parentNode||this.itemSet.dom.background.appendChild(this.dom.background),this.dom.axis.parentNode||this.itemSet.dom.axis.appendChild(this.dom.axis)},o.prototype.hide=function(){var t=this.dom.label;t.parentNode&&t.parentNode.removeChild(t);var e=this.dom.foreground;e.parentNode&&e.parentNode.removeChild(e);var i=this.dom.background;i.parentNode&&i.parentNode.removeChild(i);var o=this.dom.axis;o.parentNode&&o.parentNode.removeChild(o)},o.prototype.add=function(t){if(this.items[t.id]=t,t.setParent(this),void 0!==t.data.subgroup&&(void 0===this.subgroups[t.data.subgroup]&&(this.subgroups[t.data.subgroup]={height:0,visible:!1,index:this.subgroupIndex,items:[]},this.subgroupIndex++),this.subgroups[t.data.subgroup].items.push(t)),this.orderSubgroups(),-1==this.visibleItems.indexOf(t)){var e=this.itemSet.body.range;this._checkIfVisible(t,this.visibleItems,e)}},o.prototype.orderSubgroups=function(){if(void 0!==this.subgroupOrderer){var t=[];if("string"==typeof this.subgroupOrderer){for(var e in this.subgroups)t.push({subgroup:e,sortField:this.subgroups[e].items[0].data[this.subgroupOrderer]});t.sort(function(t,e){return t.sortField-e.sortField})}else if("function"==typeof this.subgroupOrderer){for(var e in this.subgroups)t.push(this.subgroups[e].items[0].data);t.sort(this.subgroupOrderer)}if(t.length>0)for(var i=0;it?-1:l>=t?0:1};if(e.length>0)for(s=0;sl}),1==this.checkRangedItems)for(this.checkRangedItems=!1,s=0;sl})}for(s=0;s=0&&(s=e[r],!n(s));r--)void 0===o[s.id]&&(o[s.id]=!0,i.push(s));for(r=t+1;rn;n++)t[n].top=null;for(n=0,s=t.length;s>n;n++){var r=t[n];if(r.stack&&null===r.top){r.top=i.axis;do{for(var a=null,h=0,d=t.length;d>h;h++){var l=t[h];if(null!==l.top&&l!==r&&l.stack&&e.collision(r,l,i.item,l.options.rtl)){a=l;break}}null!=a&&(r.top=a.top+a.height+i.item.vertical)}while(a)}}},e.nostack=function(t,e,i){var o,n,s;for(o=0,n=t.length;n>o;o++)if(void 0!==t[o].data.subgroup){s=e.axis;for(var r in i)i.hasOwnProperty(r)&&1==i[r].visible&&i[r].indexe.right&&t.top-o.vertical+ie.top:t.left-o.horizontal+ie.left&&t.top-o.vertical+ie.top}},function(t,e,i){function o(t,e,i){if(this.props={content:{width:0}},this.overflow=!1,this.options=i,t){if(void 0==t.start)throw new Error('Property "start" missing in item '+t.id);if(void 0==t.end)throw new Error('Property "end" missing in item '+t.id)}n.call(this,t,e,i)}var n=(i(20),i(39));o.prototype=new n(null,null,null),o.prototype.baseClassName="vis-item vis-range",o.prototype.isVisible=function(t){return this.data.startt.start},o.prototype.redraw=function(){var t=this.dom;if(t||(this.dom={},t=this.dom,t.box=document.createElement("div"),t.frame=document.createElement("div"),t.frame.className="vis-item-overflow",t.box.appendChild(t.frame),t.content=document.createElement("div"),t.content.className="vis-item-content",t.frame.appendChild(t.content),t.box["timeline-item"]=this,this.dirty=!0),!this.parent)throw new Error("Cannot redraw item: no parent attached");if(!t.box.parentNode){var e=this.parent.dom.foreground;if(!e)throw new Error("Cannot redraw item: parent has no foreground container element");e.appendChild(t.box)}if(this.displayed=!0,this.dirty){this._updateContents(this.dom.content),this._updateTitle(this.dom.box),this._updateDataAttributes(this.dom.box),this._updateStyle(this.dom.box);var i=(this.options.editable.updateTime||this.options.editable.updateGroup||this.editable===!0)&&this.editable!==!1,o=(this.data.className?" "+this.data.className:"")+(this.selected?" vis-selected":"")+(i?" vis-editable":" vis-readonly");t.box.className=this.baseClassName+o,this.overflow="hidden"!==window.getComputedStyle(t.frame).overflow,this.dom.content.style.maxWidth="none",this.props.content.width=this.dom.content.offsetWidth,this.height=this.dom.box.offsetHeight,this.dom.content.style.maxWidth="",this.dirty=!1}this._repaintDeleteButton(t.box),this._repaintDragLeft(),this._repaintDragRight()},o.prototype.show=function(){this.displayed||this.redraw()},o.prototype.hide=function(){if(this.displayed){var t=this.dom.box;t.parentNode&&t.parentNode.removeChild(t),this.displayed=!1}},o.prototype.repositionX=function(t){var e,i,o=this.parent.width,n=this.conversion.toScreen(this.data.start),s=this.conversion.toScreen(this.data.end);void 0!==t&&t!==!0||(-o>n&&(n=-o),s>2*o&&(s=2*o));var r=Math.max(s-n,1);switch(this.overflow?(this.options.rtl?this.right=n:this.left=n,this.width=r+this.props.content.width,i=this.props.content.width):(this.options.rtl?this.right=n:this.left=n,this.width=r,i=Math.min(s-n,this.props.content.width)),this.options.rtl?this.dom.box.style.right=this.right+"px":this.dom.box.style.left=this.left+"px",this.dom.box.style.width=r+"px",this.options.align){case"left":this.options.rtl?this.dom.content.style.right="0":this.dom.content.style.left="0";break;case"right":this.options.rtl?this.dom.content.style.right=Math.max(r-i,0)+"px":this.dom.content.style.left=Math.max(r-i,0)+"px";break;case"center":this.options.rtl?this.dom.content.style.right=Math.max((r-i)/2,0)+"px":this.dom.content.style.left=Math.max((r-i)/2,0)+"px";break;default:e=this.overflow?s>0?Math.max(-n,0):-i:0>n?-n:0,this.options.rtl?this.dom.content.style.right=e+"px":this.dom.content.style.left=e+"px"}},o.prototype.repositionY=function(){var t=this.options.orientation.item,e=this.dom.box;"top"==t?e.style.top=this.top+"px":e.style.top=this.parent.height-this.top-this.height+"px"},o.prototype._repaintDragLeft=function(){if(this.selected&&this.options.editable.updateTime&&!this.dom.dragLeft){var t=document.createElement("div");t.className="vis-drag-left",t.dragLeftItem=this,this.dom.box.appendChild(t),this.dom.dragLeft=t}else!this.selected&&this.dom.dragLeft&&(this.dom.dragLeft.parentNode&&this.dom.dragLeft.parentNode.removeChild(this.dom.dragLeft),this.dom.dragLeft=null)},o.prototype._repaintDragRight=function(){if(this.selected&&this.options.editable.updateTime&&!this.dom.dragRight){var t=document.createElement("div");t.className="vis-drag-right",t.dragRightItem=this,this.dom.box.appendChild(t),this.dom.dragRight=t}else!this.selected&&this.dom.dragRight&&(this.dom.dragRight.parentNode&&this.dom.dragRight.parentNode.removeChild(this.dom.dragRight),this.dom.dragRight=null)},t.exports=o},function(t,e,i){function o(t,e,i){this.id=null,this.parent=null,this.data=t,this.dom=null,this.conversion=e||{},this.options=i||{},this.selected=!1,this.displayed=!1,this.dirty=!0,this.top=null,this.right=null,this.left=null,this.width=null,this.height=null,this.editable=null,this.data&&this.data.hasOwnProperty("editable")&&"boolean"==typeof this.data.editable&&(this.editable=t.editable)}var n=i(20),s=i(1);o.prototype.stack=!0,o.prototype.select=function(){this.selected=!0,this.dirty=!0,this.displayed&&this.redraw()},o.prototype.unselect=function(){this.selected=!1,this.dirty=!0,this.displayed&&this.redraw()},o.prototype.setData=function(t){var e=void 0!=t.group&&this.data.group!=t.group;e&&this.parent.itemSet._moveToGroup(this,t.group),t.hasOwnProperty("editable")&&"boolean"==typeof t.editable&&(this.editable=t.editable),this.data=t,this.dirty=!0,this.displayed&&this.redraw()},o.prototype.setParent=function(t){this.displayed?(this.hide(),this.parent=t,this.parent&&this.show()):this.parent=t},o.prototype.isVisible=function(t){return!1},o.prototype.show=function(){return!1},o.prototype.hide=function(){return!1},o.prototype.redraw=function(){},o.prototype.repositionX=function(){},o.prototype.repositionY=function(){},o.prototype._repaintDeleteButton=function(t){var e=(this.options.editable.remove||this.data.editable===!0)&&this.data.editable!==!1;if(this.selected&&e&&!this.dom.deleteButton){var i=this,o=document.createElement("div");this.options.rtl?o.className="vis-delete-rtl":o.className="vis-delete",o.title="Delete this item",new n(o).on("tap",function(t){t.stopPropagation(),i.parent.removeFromDataSet(i)}),t.appendChild(o),this.dom.deleteButton=o}else!this.selected&&this.dom.deleteButton&&(this.dom.deleteButton.parentNode&&this.dom.deleteButton.parentNode.removeChild(this.dom.deleteButton),this.dom.deleteButton=null)},o.prototype._updateContents=function(t){var e;if(this.options.template){var i=this.parent.itemSet.itemsData.get(this.id);e=this.options.template(i)}else e=this.data.content;var o=this._contentToString(this.content)!==this._contentToString(e);if(o){if(e instanceof Element)t.innerHTML="",t.appendChild(e);else if(void 0!=e)t.innerHTML=e;else if("background"!=this.data.type||void 0!==this.data.content)throw new Error('Property "content" missing in item '+this.id);this.content=e}},o.prototype._updateTitle=function(t){null!=this.data.title?t.title=this.data.title||"":t.removeAttribute("vis-title")},o.prototype._updateDataAttributes=function(t){if(this.options.dataAttributes&&this.options.dataAttributes.length>0){var e=[];if(Array.isArray(this.options.dataAttributes))e=this.options.dataAttributes;else{if("all"!=this.options.dataAttributes)return;e=Object.keys(this.data)}for(var i=0;in;n++){var r=this.visibleItems[n];r.repositionY(e)}return o},o.prototype.show=function(){this.dom.background.parentNode||this.itemSet.dom.background.appendChild(this.dom.background)},t.exports=o},function(t,e,i){function o(t,e,i){if(this.props={dot:{width:0,height:0},line:{width:0,height:0}},this.options=i,t&&void 0==t.start)throw new Error('Property "start" missing in item '+t);n.call(this,t,e,i)}var n=i(39);i(1);o.prototype=new n(null,null,null),o.prototype.isVisible=function(t){var e=(t.end-t.start)/4;return this.data.start>t.start-e&&this.data.startt.start-e&&this.data.startt.start},o.prototype.redraw=function(){var t=this.dom;if(t||(this.dom={},t=this.dom,t.box=document.createElement("div"),t.frame=document.createElement("div"),t.frame.className="vis-item-overflow",t.box.appendChild(t.frame),t.content=document.createElement("div"),t.content.className="vis-item-content",t.frame.appendChild(t.content),this.dirty=!0),!this.parent)throw new Error("Cannot redraw item: no parent attached");if(!t.box.parentNode){var e=this.parent.dom.background;if(!e)throw new Error("Cannot redraw item: parent has no background container element");e.appendChild(t.box)}if(this.displayed=!0,this.dirty){this._updateContents(this.dom.content),this._updateTitle(this.dom.content),this._updateDataAttributes(this.dom.content),this._updateStyle(this.dom.box);var i=(this.data.className?" "+this.data.className:"")+(this.selected?" vis-selected":"");t.box.className=this.baseClassName+i,this.overflow="hidden"!==window.getComputedStyle(t.content).overflow,this.props.content.width=this.dom.content.offsetWidth,this.height=0,this.dirty=!1}},o.prototype.show=r.prototype.show,o.prototype.hide=r.prototype.hide,o.prototype.repositionX=r.prototype.repositionX,o.prototype.repositionY=function(t){var e="top"===this.options.orientation.item;this.dom.content.style.top=e?"":"0",this.dom.content.style.bottom=e?"0":"";var i;if(void 0!==this.data.subgroup){var o=this.data.subgroup,n=this.parent.subgroups,r=n[o].index;if(1==e){i=this.parent.subgroups[o].height+t.item.vertical,i+=0==r?t.axis-.5*t.item.vertical:0;var a=this.parent.top;for(var h in n)n.hasOwnProperty(h)&&1==n[h].visible&&n[h].indexr&&(a+=l)}i=this.parent.subgroups[o].height+t.item.vertical,this.dom.box.style.top=this.parent.height-d+a+"px",this.dom.box.style.bottom=""}}else this.parent instanceof s?(i=Math.max(this.parent.height,this.parent.itemSet.body.domProps.center.height,this.parent.itemSet.body.domProps.centerContainer.height),this.dom.box.style.top=e?"0":"",this.dom.box.style.bottom=e?"":"0"):(i=this.parent.height,this.dom.box.style.top=this.parent.top+"px",this.dom.box.style.bottom="");this.dom.box.style.height=i+"px"},t.exports=o},function(t,e,i){function o(t,e){this.dom={foreground:null,lines:[],majorTexts:[],minorTexts:[],redundant:{lines:[],majorTexts:[],minorTexts:[]}},this.props={range:{start:0,end:0,minimumStep:0},lineTop:0},this.defaultOptions={orientation:{axis:"bottom"},showMinorLabels:!0,showMajorLabels:!0,maxMinorChars:7,format:a.FORMAT,moment:d,timeAxis:null},this.options=s.extend({},this.defaultOptions),this.body=t,this._create(),this.setOptions(e)}var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=i(1),r=i(31),a=i(35),h=i(32),d=i(2);o.prototype=new r,o.prototype.setOptions=function(t){t&&(s.selectiveExtend(["showMinorLabels","showMajorLabels","maxMinorChars","hiddenDates","timeAxis","moment","rtl"],this.options,t),s.selectiveDeepExtend(["format"],this.options,t),"orientation"in t&&("string"==typeof t.orientation?this.options.orientation.axis=t.orientation:"object"===n(t.orientation)&&"axis"in t.orientation&&(this.options.orientation.axis=t.orientation.axis)),"locale"in t&&("function"==typeof d.locale?d.locale(t.locale):d.lang(t.locale)))},o.prototype._create=function(){this.dom.foreground=document.createElement("div"),this.dom.background=document.createElement("div"),this.dom.foreground.className="vis-time-axis vis-foreground",this.dom.background.className="vis-time-axis vis-background"},o.prototype.destroy=function(){this.dom.foreground.parentNode&&this.dom.foreground.parentNode.removeChild(this.dom.foreground),this.dom.background.parentNode&&this.dom.background.parentNode.removeChild(this.dom.background),this.body=null},o.prototype.redraw=function(){var t=this.props,e=this.dom.foreground,i=this.dom.background,o="top"==this.options.orientation.axis?this.body.dom.top:this.body.dom.bottom,n=e.parentNode!==o;this._calculateCharSize();var s=this.options.showMinorLabels&&"none"!==this.options.orientation.axis,r=this.options.showMajorLabels&&"none"!==this.options.orientation.axis;t.minorLabelHeight=s?t.minorCharHeight:0,t.majorLabelHeight=r?t.majorCharHeight:0,t.height=t.minorLabelHeight+t.majorLabelHeight,t.width=e.offsetWidth,t.minorLineHeight=this.body.domProps.root.height-t.majorLabelHeight-("top"==this.options.orientation.axis?this.body.domProps.bottom.height:this.body.domProps.top.height),t.minorLineWidth=1,t.majorLineHeight=t.minorLineHeight+t.majorLabelHeight,t.majorLineWidth=1;var a=e.nextSibling,h=i.nextSibling;return e.parentNode&&e.parentNode.removeChild(e),i.parentNode&&i.parentNode.removeChild(i),e.style.height=this.props.height+"px",this._repaintLabels(),a?o.insertBefore(e,a):o.appendChild(e),h?this.body.dom.backgroundVertical.insertBefore(i,h):this.body.dom.backgroundVertical.appendChild(i),this._isResized()||n},o.prototype._repaintLabels=function(){var t=this.options.orientation.axis,e=s.convert(this.body.range.start,"Number"),i=s.convert(this.body.range.end,"Number"),o=this.body.util.toTime((this.props.minorCharWidth||10)*this.options.maxMinorChars).valueOf(),n=o-h.getHiddenDurationBefore(this.options.moment,this.body.hiddenDates,this.body.range,o);n-=this.body.util.toTime(0).valueOf();var r=new a(new Date(e),new Date(i),n,this.body.hiddenDates);r.setMoment(this.options.moment),this.options.format&&r.setFormat(this.options.format),this.options.timeAxis&&r.setScale(this.options.timeAxis),this.step=r;var d=this.dom;d.redundant.lines=d.lines,d.redundant.majorTexts=d.majorTexts,d.redundant.minorTexts=d.minorTexts,d.lines=[],d.majorTexts=[],d.minorTexts=[];var c,u,p,f,m,v,g,y,b,w,_=0,x=void 0,k=0,O=1e3;for(r.start(),u=r.getCurrent(),f=this.body.util.toScreen(u);r.hasNext()&&O>k;){k++,m=r.isMajor(),w=r.getClassName(),b=r.getLabelMinor(),c=u,p=f,r.next(),u=r.getCurrent(),v=r.isMajor(),f=this.body.util.toScreen(u),g=_,_=f-p;var M=_>=.4*g;if(this.options.showMinorLabels&&M){var D=this._repaintMinorText(p,b,t,w);D.style.width=_+"px"}m&&this.options.showMajorLabels?(p>0&&(void 0==x&&(x=p),D=this._repaintMajorText(p,r.getLabelMajor(),t,w)),y=this._repaintMajorLine(p,_,t,w)):M?y=this._repaintMinorLine(p,_,t,w):y&&(y.style.width=parseInt(y.style.width)+_+"px")}if(k!==O||l||(console.warn("Something is wrong with the Timeline scale. Limited drawing of grid lines to "+O+" lines."),l=!0),this.options.showMajorLabels){var S=this.body.util.toTime(0),C=r.getLabelMajor(S),T=C.length*(this.props.majorCharWidth||10)+10;(void 0==x||x>T)&&this._repaintMajorText(0,C,t,w)}s.forEach(this.dom.redundant,function(t){for(;t.length;){var e=t.pop();e&&e.parentNode&&e.parentNode.removeChild(e)}})},o.prototype._repaintMinorText=function(t,e,i,o){var n=this.dom.redundant.minorTexts.shift();if(!n){var s=document.createTextNode("");n=document.createElement("div"),n.appendChild(s),this.dom.foreground.appendChild(n)}return this.dom.minorTexts.push(n),n.childNodes[0].nodeValue=e,n.style.top="top"==i?this.props.majorLabelHeight+"px":"0",this.options.rtl?(n.style.left="",n.style.right=t+"px"):n.style.left=t+"px",n.className="vis-text vis-minor "+o,n},o.prototype._repaintMajorText=function(t,e,i,o){var n=this.dom.redundant.majorTexts.shift();if(!n){var s=document.createTextNode(e);n=document.createElement("div"),n.appendChild(s),this.dom.foreground.appendChild(n)}return this.dom.majorTexts.push(n),n.childNodes[0].nodeValue=e,n.className="vis-text vis-major "+o,n.style.top="top"==i?"0":this.props.minorLabelHeight+"px",this.options.rtl?(n.style.left="",n.style.right=t+"px"):n.style.left=t+"px",n},o.prototype._repaintMinorLine=function(t,e,i,o){var n=this.dom.redundant.lines.shift();n||(n=document.createElement("div"),this.dom.background.appendChild(n)),this.dom.lines.push(n);var s=this.props;return"top"==i?n.style.top=s.majorLabelHeight+"px":n.style.top=this.body.domProps.top.height+"px",n.style.height=s.minorLineHeight+"px",this.options.rtl?(n.style.left="",n.style.right=t-s.minorLineWidth/2+"px",n.className="vis-grid vis-vertical-rtl vis-minor "+o):(n.style.left=t-s.minorLineWidth/2+"px",n.className="vis-grid vis-vertical vis-minor "+o),n.style.width=e+"px",n},o.prototype._repaintMajorLine=function(t,e,i,o){var n=this.dom.redundant.lines.shift();n||(n=document.createElement("div"),this.dom.background.appendChild(n)),this.dom.lines.push(n);var s=this.props;return"top"==i?n.style.top="0":n.style.top=this.body.domProps.top.height+"px",this.options.rtl?(n.style.left="",n.style.right=t-s.majorLineWidth/2+"px",n.className="vis-grid vis-vertical-rtl vis-major "+o):(n.style.left=t-s.majorLineWidth/2+"px",n.className="vis-grid vis-vertical vis-major "+o),n.style.height=s.majorLineHeight+"px",n.style.width=e+"px",n},o.prototype._calculateCharSize=function(){this.dom.measureCharMinor||(this.dom.measureCharMinor=document.createElement("DIV"),this.dom.measureCharMinor.className="vis-text vis-minor vis-measure",this.dom.measureCharMinor.style.position="absolute",this.dom.measureCharMinor.appendChild(document.createTextNode("0")),this.dom.foreground.appendChild(this.dom.measureCharMinor)),this.props.minorCharHeight=this.dom.measureCharMinor.clientHeight,this.props.minorCharWidth=this.dom.measureCharMinor.clientWidth,this.dom.measureCharMajor||(this.dom.measureCharMajor=document.createElement("DIV"),this.dom.measureCharMajor.className="vis-text vis-major vis-measure",this.dom.measureCharMajor.style.position="absolute",this.dom.measureCharMajor.appendChild(document.createTextNode("0")),this.dom.foreground.appendChild(this.dom.measureCharMajor)),this.props.majorCharHeight=this.dom.measureCharMajor.clientHeight,this.props.majorCharWidth=this.dom.measureCharMajor.clientWidth};var l=!1;t.exports=o},function(t,e,i){function o(t){this.active=!1,this.dom={container:t},this.dom.overlay=document.createElement("div"),this.dom.overlay.className="vis-overlay",this.dom.container.appendChild(this.dom.overlay),this.hammer=a(this.dom.overlay),this.hammer.on("tap",this._onTapOverlay.bind(this));var e=this,i=["tap","doubletap","press","pinch","pan","panstart","panmove","panend"];i.forEach(function(t){e.hammer.on(t,function(t){t.stopPropagation()})}),document&&document.body&&(this.onClick=function(i){n(i.target,t)||e.deactivate()},document.body.addEventListener("click",this.onClick)),void 0!==this.keycharm&&this.keycharm.destroy(),this.keycharm=s(),this.escListener=this.deactivate.bind(this)}function n(t,e){for(;t;){if(t===e)return!0;t=t.parentNode}return!1}var s=i(23),r=i(13),a=i(20),h=i(1);r(o.prototype),o.current=null,o.prototype.destroy=function(){this.deactivate(),this.dom.overlay.parentNode.removeChild(this.dom.overlay),this.onClick&&document.body.removeEventListener("click",this.onClick),this.hammer.destroy(),this.hammer=null},o.prototype.activate=function(){o.current&&o.current.deactivate(),o.current=this,this.active=!0,this.dom.overlay.style.display="none",h.addClassName(this.dom.container,"vis-active"),this.emit("change"),this.emit("activate"),this.keycharm.bind("esc",this.escListener)},o.prototype.deactivate=function(){this.active=!1,this.dom.overlay.style.display="",h.removeClassName(this.dom.container,"vis-active"),this.keycharm.unbind("esc",this.escListener),this.emit("change"),this.emit("deactivate")},o.prototype._onTapOverlay=function(t){this.activate(),t.stopPropagation()},t.exports=o},function(t,e,i){function o(t,e){this.body=t,this.defaultOptions={moment:a,locales:h,locale:"en",id:void 0,title:void 0},this.options=s.extend({},this.defaultOptions),e&&e.time?this.customTime=e.time:this.customTime=new Date,this.eventParams={},this.setOptions(e),this._create()}var n=i(20),s=i(1),r=i(31),a=i(2),h=i(47);o.prototype=new r,o.prototype.setOptions=function(t){t&&s.selectiveExtend(["moment","locale","locales","id"],this.options,t)},o.prototype._create=function(){var t=document.createElement("div");t["custom-time"]=this,t.className="vis-custom-time "+(this.options.id||""),t.style.position="absolute",t.style.top="0px",t.style.height="100%",this.bar=t;var e=document.createElement("div");e.style.position="relative",e.style.top="0px",e.style.left="-10px",e.style.height="100%",e.style.width="20px",t.appendChild(e),this.hammer=new n(e),this.hammer.on("panstart",this._onDragStart.bind(this)),this.hammer.on("panmove",this._onDrag.bind(this)),this.hammer.on("panend",this._onDragEnd.bind(this)),this.hammer.get("pan").set({threshold:5,direction:n.DIRECTION_HORIZONTAL})},o.prototype.destroy=function(){this.hide(),this.hammer.destroy(),this.hammer=null,this.body=null},o.prototype.redraw=function(){var t=this.body.dom.backgroundVertical;this.bar.parentNode!=t&&(this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),t.appendChild(this.bar));var e=this.body.util.toScreen(this.customTime),i=this.options.locales[this.options.locale];i||(this.warned||(console.log("WARNING: options.locales['"+this.options.locale+"'] not found. See http://visjs.org/docs/timeline.html#Localization"),this.warned=!0),i=this.options.locales.en);var o=this.options.title;return void 0===o&&(o=i.time+": "+this.options.moment(this.customTime).format("dddd, MMMM Do YYYY, H:mm:ss"),o=o.charAt(0).toUpperCase()+o.substring(1)),this.bar.style.left=e+"px",this.bar.title=o,!1},o.prototype.hide=function(){this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar)},o.prototype.setCustomTime=function(t){this.customTime=s.convert(t,"Date"),this.redraw()},o.prototype.getCustomTime=function(){return new Date(this.customTime.valueOf())},o.prototype.setCustomTitle=function(t){this.options.title=t},o.prototype._onDragStart=function(t){this.eventParams.dragging=!0,this.eventParams.customTime=this.customTime,t.stopPropagation()},o.prototype._onDrag=function(t){if(this.eventParams.dragging){var e=this.body.util.toScreen(this.eventParams.customTime)+t.deltaX,i=this.body.util.toTime(e);this.setCustomTime(i),this.body.emitter.emit("timechange",{id:this.options.id,time:new Date(this.customTime.valueOf()) +}),t.stopPropagation()}},o.prototype._onDragEnd=function(t){this.eventParams.dragging&&(this.body.emitter.emit("timechanged",{id:this.options.id,time:new Date(this.customTime.valueOf())}),t.stopPropagation())},o.customTimeFromTarget=function(t){for(var e=t.target;e;){if(e.hasOwnProperty("custom-time"))return e["custom-time"];e=e.parentNode}return null},t.exports=o},function(t,e){e.en={current:"current",time:"time"},e.en_EN=e.en,e.en_US=e.en,e.nl={current:"huidige",time:"tijd"},e.nl_NL=e.nl,e.nl_BE=e.nl},function(t,e,i){function o(t,e){this.body=t,this.defaultOptions={rtl:!1,showCurrentTime:!0,moment:r,locales:a,locale:"en"},this.options=n.extend({},this.defaultOptions),this.offset=0,this._create(),this.setOptions(e)}var n=i(1),s=i(31),r=i(2),a=i(47);o.prototype=new s,o.prototype._create=function(){var t=document.createElement("div");t.className="vis-current-time",t.style.position="absolute",t.style.top="0px",t.style.height="100%",this.bar=t},o.prototype.destroy=function(){this.options.showCurrentTime=!1,this.redraw(),this.body=null},o.prototype.setOptions=function(t){t&&n.selectiveExtend(["rtl","showCurrentTime","moment","locale","locales"],this.options,t)},o.prototype.redraw=function(){if(this.options.showCurrentTime){var t=this.body.dom.backgroundVertical;this.bar.parentNode!=t&&(this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),t.appendChild(this.bar),this.start());var e=this.options.moment((new Date).valueOf()+this.offset),i=this.body.util.toScreen(e),o=this.options.locales[this.options.locale];o||(this.warned||(console.log("WARNING: options.locales['"+this.options.locale+"'] not found. See http://visjs.org/docs/timeline/#Localization"),this.warned=!0),o=this.options.locales.en);var n=o.current+" "+o.time+": "+e.format("dddd, MMMM Do YYYY, H:mm:ss");n=n.charAt(0).toUpperCase()+n.substring(1),this.options.rtl?this.bar.style.right=i+"px":this.bar.style.left=i+"px",this.bar.title=n}else this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),this.stop();return!1},o.prototype.start=function(){function t(){e.stop();var i=e.body.range.conversion(e.body.domProps.center.width).scale,o=1/i/10;30>o&&(o=30),o>1e3&&(o=1e3),e.redraw(),e.body.emitter.emit("currentTimeTick"),e.currentTimeTimer=setTimeout(t,o)}var e=this;t()},o.prototype.stop=function(){void 0!==this.currentTimeTimer&&(clearTimeout(this.currentTimeTimer),delete this.currentTimeTimer)},o.prototype.setCurrentTime=function(t){var e=n.convert(t,"Date").valueOf(),i=(new Date).valueOf();this.offset=e-i,this.redraw()},o.prototype.getCurrentTime=function(){return new Date((new Date).valueOf()+this.offset)},t.exports=o},function(t,e){Object.defineProperty(e,"__esModule",{value:!0});var i="string",o="boolean",n="number",s="array",r="date",a="object",h="dom",d="moment",l="any",c={configure:{enabled:{"boolean":o},filter:{"boolean":o,"function":"function"},container:{dom:h},__type__:{object:a,"boolean":o,"function":"function"}},align:{string:i},rtl:{"boolean":o,undefined:"undefined"},autoResize:{"boolean":o},throttleRedraw:{number:n},clickToUse:{"boolean":o},dataAttributes:{string:i,array:s},editable:{add:{"boolean":o,undefined:"undefined"},remove:{"boolean":o,undefined:"undefined"},updateGroup:{"boolean":o,undefined:"undefined"},updateTime:{"boolean":o,undefined:"undefined"},__type__:{"boolean":o,object:a}},end:{number:n,date:r,string:i,moment:d},format:{minorLabels:{millisecond:{string:i,undefined:"undefined"},second:{string:i,undefined:"undefined"},minute:{string:i,undefined:"undefined"},hour:{string:i,undefined:"undefined"},weekday:{string:i,undefined:"undefined"},day:{string:i,undefined:"undefined"},month:{string:i,undefined:"undefined"},year:{string:i,undefined:"undefined"},__type__:{object:a}},majorLabels:{millisecond:{string:i,undefined:"undefined"},second:{string:i,undefined:"undefined"},minute:{string:i,undefined:"undefined"},hour:{string:i,undefined:"undefined"},weekday:{string:i,undefined:"undefined"},day:{string:i,undefined:"undefined"},month:{string:i,undefined:"undefined"},year:{string:i,undefined:"undefined"},__type__:{object:a}},__type__:{object:a}},moment:{"function":"function"},groupOrder:{string:i,"function":"function"},groupEditable:{add:{"boolean":o,undefined:"undefined"},remove:{"boolean":o,undefined:"undefined"},order:{"boolean":o,undefined:"undefined"},__type__:{"boolean":o,object:a}},groupOrderSwap:{"function":"function"},height:{string:i,number:n},hiddenDates:{start:{date:r,number:n,string:i,moment:d},end:{date:r,number:n,string:i,moment:d},repeat:{string:i},__type__:{object:a,array:s}},itemsAlwaysDraggable:{"boolean":o},locale:{string:i},locales:{__any__:{any:l},__type__:{object:a}},margin:{axis:{number:n},item:{horizontal:{number:n,undefined:"undefined"},vertical:{number:n,undefined:"undefined"},__type__:{object:a,number:n}},__type__:{object:a,number:n}},max:{date:r,number:n,string:i,moment:d},maxHeight:{number:n,string:i},maxMinorChars:{number:n},min:{date:r,number:n,string:i,moment:d},minHeight:{number:n,string:i},moveable:{"boolean":o},multiselect:{"boolean":o},multiselectPerGroup:{"boolean":o},onAdd:{"function":"function"},onUpdate:{"function":"function"},onMove:{"function":"function"},onMoving:{"function":"function"},onRemove:{"function":"function"},onAddGroup:{"function":"function"},onMoveGroup:{"function":"function"},onRemoveGroup:{"function":"function"},order:{"function":"function"},orientation:{axis:{string:i,undefined:"undefined"},item:{string:i,undefined:"undefined"},__type__:{string:i,object:a}},selectable:{"boolean":o},showCurrentTime:{"boolean":o},showMajorLabels:{"boolean":o},showMinorLabels:{"boolean":o},stack:{"boolean":o},snap:{"function":"function","null":"null"},start:{date:r,number:n,string:i,moment:d},template:{"function":"function"},groupTemplate:{"function":"function"},timeAxis:{scale:{string:i,undefined:"undefined"},step:{number:n,undefined:"undefined"},__type__:{object:a}},type:{string:i},width:{string:i,number:n},zoomable:{"boolean":o},zoomKey:{string:["ctrlKey","altKey","metaKey",""]},zoomMax:{number:n},zoomMin:{number:n},__type__:{object:a}},u={global:{align:["center","left","right"],direction:!1,autoResize:!0,throttleRedraw:[10,0,1e3,10],clickToUse:!1,editable:{add:!1,remove:!1,updateGroup:!1,updateTime:!1},end:"",format:{minorLabels:{millisecond:"SSS",second:"s",minute:"HH:mm",hour:"HH:mm",weekday:"ddd D",day:"D",month:"MMM",year:"YYYY"},majorLabels:{millisecond:"HH:mm:ss",second:"D MMMM HH:mm",minute:"ddd D MMMM",hour:"ddd D MMMM",weekday:"MMMM YYYY",day:"MMMM YYYY",month:"YYYY",year:""}},groupsDraggable:!1,height:"",locale:"",margin:{axis:[20,0,100,1],item:{horizontal:[10,0,100,1],vertical:[10,0,100,1]}},max:"",maxHeight:"",maxMinorChars:[7,0,20,1],min:"",minHeight:"",moveable:!1,multiselect:!1,multiselectPerGroup:!1,orientation:{axis:["both","bottom","top"],item:["bottom","top"]},selectable:!0,showCurrentTime:!1,showMajorLabels:!0,showMinorLabels:!0,stack:!0,start:"",type:["box","point","range","background"],width:"100%",zoomable:!0,zoomKey:["ctrlKey","altKey","metaKey",""],zoomMax:[31536e10,10,31536e10,1],zoomMin:[10,10,31536e10,1]}};e.allOptions=c,e.configureOptions=u},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e,i,o){if(!(Array.isArray(i)||i instanceof c||i instanceof u)&&i instanceof Object){var n=o;o=i,i=n}var s=this;this.defaultOptions={start:null,end:null,autoResize:!0,orientation:{axis:"bottom",item:"bottom"},moment:d,width:null,height:null,maxHeight:null,minHeight:null},this.options=l.deepExtend({},this.defaultOptions),this._create(t),this.components=[],this.body={dom:this.dom,domProps:this.props,emitter:{on:this.on.bind(this),off:this.off.bind(this),emit:this.emit.bind(this)},hiddenDates:[],util:{toScreen:s._toScreen.bind(s),toGlobalScreen:s._toGlobalScreen.bind(s),toTime:s._toTime.bind(s),toGlobalTime:s._toGlobalTime.bind(s)}},this.range=new p(this.body),this.components.push(this.range),this.body.range=this.range,this.timeAxis=new m(this.body),this.components.push(this.timeAxis),this.currentTime=new v(this.body),this.components.push(this.currentTime),this.linegraph=new y(this.body),this.components.push(this.linegraph),this.itemsData=null,this.groupsData=null,this.on("tap",function(t){s.emit("click",s.getEventProperties(t))}),this.on("doubletap",function(t){s.emit("doubleClick",s.getEventProperties(t))}),this.dom.root.oncontextmenu=function(t){s.emit("contextmenu",s.getEventProperties(t))},o&&this.setOptions(o),i&&this.setGroups(i),e&&this.setItems(e),this._redraw()}var s=i(26),r=o(s),a=i(29),h=o(a),d=(i(13),i(20),i(2)),l=i(1),c=i(9),u=i(11),p=i(30),f=i(33),m=i(44),v=i(48),g=i(46),y=i(51),b=i(29).printStyle,w=i(59).allOptions,_=i(59).configureOptions;n.prototype=new f,n.prototype.setOptions=function(t){var e=h["default"].validate(t,w);e===!0&&console.log("%cErrors have been found in the supplied options object.",b),f.prototype.setOptions.call(this,t)},n.prototype.setItems=function(t){var e,i=null==this.itemsData;if(e=t?t instanceof c||t instanceof u?t:new c(t,{type:{start:"Date",end:"Date"}}):null,this.itemsData=e,this.linegraph&&this.linegraph.setItems(e),i)if(void 0!=this.options.start||void 0!=this.options.end){var o=void 0!=this.options.start?this.options.start:null,n=void 0!=this.options.end?this.options.end:null;this.setWindow(o,n,{animation:!1})}else this.fit({animation:!1})},n.prototype.setGroups=function(t){var e;e=t?t instanceof c||t instanceof u?t:new c(t):null,this.groupsData=e,this.linegraph.setGroups(e)},n.prototype.getLegend=function(t,e,i){return void 0===e&&(e=15),void 0===i&&(i=15),void 0!==this.linegraph.groups[t]?this.linegraph.groups[t].getLegend(e,i):"cannot find group:'"+t+"'"},n.prototype.isGroupVisible=function(t){return void 0!==this.linegraph.groups[t]?this.linegraph.groups[t].visible&&(void 0===this.linegraph.options.groups.visibility[t]||1==this.linegraph.options.groups.visibility[t]):!1},n.prototype.getDataRange=function(){var t=null,e=null;for(var i in this.linegraph.groups)if(this.linegraph.groups.hasOwnProperty(i)&&1==this.linegraph.groups[i].visible)for(var o=0;os?s:t,e=null==e?s:s>e?s:e}return{min:null!=t?new Date(t):null,max:null!=e?new Date(e):null}},n.prototype.getEventProperties=function(t){var e=t.center?t.center.x:t.clientX,i=t.center?t.center.y:t.clientY,o=e-l.getAbsoluteLeft(this.dom.centerContainer),n=i-l.getAbsoluteTop(this.dom.centerContainer),s=this._toTime(o),r=g.customTimeFromTarget(t),a=l.getTarget(t),h=null;l.hasParent(a,this.timeAxis.dom.foreground)?h="axis":this.timeAxis2&&l.hasParent(a,this.timeAxis2.dom.foreground)?h="axis":l.hasParent(a,this.linegraph.yAxisLeft.dom.frame)?h="data-axis":l.hasParent(a,this.linegraph.yAxisRight.dom.frame)?h="data-axis":l.hasParent(a,this.linegraph.legendLeft.dom.frame)?h="legend":l.hasParent(a,this.linegraph.legendRight.dom.frame)?h="legend":null!=r?h="custom-time":l.hasParent(a,this.currentTime.bar)?h="current-time":l.hasParent(a,this.dom.center)&&(h="background");var d=[],c=this.linegraph.yAxisLeft,u=this.linegraph.yAxisRight;return c.hidden||d.push(c.screenToValue(n)),u.hidden||d.push(u.screenToValue(n)),{event:t,what:h,pageX:t.srcEvent?t.srcEvent.pageX:t.pageX,pageY:t.srcEvent?t.srcEvent.pageY:t.pageY,x:o,y:n,time:s,value:d}},n.prototype._createConfigurator=function(){return new r["default"](this,this.dom.container,_)},t.exports=n},function(t,e,i){function o(t,e){this.id=s.randomUUID(),this.body=t,this.defaultOptions={yAxisOrientation:"left",defaultGroup:"default",sort:!0,sampling:!0,stack:!1,graphHeight:"400px",shaded:{enabled:!1,orientation:"bottom"},style:"line",barChart:{width:50,sideBySide:!1,align:"center"},interpolation:{enabled:!0,parametrization:"centripetal",alpha:.5},drawPoints:{enabled:!0,size:6,style:"square"},dataAxis:{},legend:{},groups:{visibility:{}}},this.options=s.extend({},this.defaultOptions),this.dom={},this.props={},this.hammer=null,this.groups={},this.abortedGraphUpdate=!1,this.updateSVGheight=!1,this.updateSVGheightOnResize=!1,this.forceGraphUpdate=!0;var i=this;this.itemsData=null,this.groupsData=null,this.itemListeners={add:function(t,e,o){i._onAdd(e.items)},update:function(t,e,o){i._onUpdate(e.items)},remove:function(t,e,o){i._onRemove(e.items)}},this.groupListeners={add:function(t,e,o){i._onAddGroups(e.items)},update:function(t,e,o){i._onUpdateGroups(e.items)},remove:function(t,e,o){i._onRemoveGroups(e.items)}},this.items={},this.selection=[],this.lastStart=this.body.range.start,this.touchParams={},this.svgElements={},this.setOptions(e),this.groupsUsingDefaultStyles=[0],this.body.emitter.on("rangechanged",function(){i.lastStart=i.body.range.start,i.svg.style.left=s.option.asSize(-i.props.width),i.forceGraphUpdate=!0,i.redraw.call(i)}),this._create(),this.framework={svg:this.svg,svgElements:this.svgElements,options:this.options,groups:this.groups}}var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=i(1),r=i(8),a=i(9),h=i(11),d=i(31),l=i(52),c=i(54),u=i(58),p=i(55),f=i(57),m=i(56),v="__ungrouped__";o.prototype=new d,o.prototype._create=function(){var t=document.createElement("div");t.className="vis-line-graph",this.dom.frame=t,this.svg=document.createElementNS("http://www.w3.org/2000/svg","svg"),this.svg.style.position="relative",this.svg.style.height=(""+this.options.graphHeight).replace("px","")+"px",this.svg.style.display="block",t.appendChild(this.svg),this.options.dataAxis.orientation="left",this.yAxisLeft=new l(this.body,this.options.dataAxis,this.svg,this.options.groups),this.options.dataAxis.orientation="right",this.yAxisRight=new l(this.body,this.options.dataAxis,this.svg,this.options.groups),delete this.options.dataAxis.orientation,this.legendLeft=new u(this.body,this.options.legend,"left",this.options.groups),this.legendRight=new u(this.body,this.options.legend,"right",this.options.groups),this.show()},o.prototype.setOptions=function(t){if(t){var e=["sampling","defaultGroup","stack","height","graphHeight","yAxisOrientation","style","barChart","dataAxis","sort","groups"];void 0===t.graphHeight&&void 0!==t.height?(this.updateSVGheight=!0,this.updateSVGheightOnResize=!0):void 0!==this.body.domProps.centerContainer.height&&void 0!==t.graphHeight&&parseInt((t.graphHeight+"").replace("px",""))i?-1:1});for(var o=new Array(t.length),n=0;n0){var h={};for(this._getRelevantData(a,h,n,s),this._applySampling(a,h),e=0;e0)switch(t.options.style){case"line":l.hasOwnProperty(a[e])||(l[a[e]]=f.calcPath(h[a[e]],t)),f.draw(l[a[e]],t,this.framework);case"point":case"points":"point"!=t.options.style&&"points"!=t.options.style&&1!=t.options.drawPoints.enabled||m.draw(h[a[e]],t,this.framework);break;case"bar":}}}return r.cleanupElements(this.svgElements),!1},o.prototype._stack=function(t,e){var i,o,n,s,r;i=0;for(var a=0;at[a].x){r=e[h],s=0==h?r:e[h-1],i=h;break}}void 0===r&&(s=e[e.length-1],r=e[e.length-1]),o=r.x-s.x,n=r.y-s.y,0==o?t[a].y=t[a].orginalY+r.y:t[a].y=t[a].orginalY+n/o*(t[a].x-s.x)+s.y}},o.prototype._getRelevantData=function(t,e,i,o){var n,r,a,h;if(t.length>0)for(r=0;rt?-1:1},c=Math.max(0,s.binarySearchValue(d,i,"x","before",l)),u=Math.min(d.length,s.binarySearchValue(d,o,"x","after",l)+1);0>=u&&(u=d.length);var p=new Array(u-c);for(a=c;u>a;a++)h=n.itemsData[a],p[a-c]=h;e[t[r]]=p}else e[t[r]]=n.itemsData}},o.prototype._applySampling=function(t,e){var i;if(t.length>0)for(var o=0;o0){var s=1,r=n.length,a=this.body.util.toGlobalScreen(n[n.length-1].x)-this.body.util.toGlobalScreen(n[0].x),h=r/a;s=Math.min(Math.ceil(.2*r),Math.max(1,Math.round(h)));for(var d=new Array(r),l=0;r>l;l+=s){var c=Math.round(l/s);d[c]=n[l]}e[t[o]]=d.splice(0,Math.round(r/s))}}},o.prototype._getYRanges=function(t,e,i){var o,n,s,r,a=[],h=[];if(t.length>0){for(s=0;s0&&(n=this.groups[t[s]],r.stack===!0&&"bar"===r.style?"left"===r.yAxisOrientation?a=a.concat(n.getItems()):h=h.concat(n.getItems()):i[t[s]]=n.getYRange(o,t[s]));p.getStackedYRange(a,i,t,"__barStackLeft","left"),p.getStackedYRange(h,i,t,"__barStackRight","right")}},o.prototype._updateYAxis=function(t,e){var i,o,n=!1,s=!1,r=!1,a=1e9,h=1e9,d=-1e9,l=-1e9;if(t.length>0){for(var c=0;ci?i:a,d=o>d?o:d):(r=!0,h=h>i?i:h,l=o>l?o:l));1==s&&this.yAxisLeft.setRange(a,d),1==r&&this.yAxisRight.setRange(h,l)}n=this._toggleAxisVisiblity(s,this.yAxisLeft)||n,n=this._toggleAxisVisiblity(r,this.yAxisRight)||n,1==r&&1==s?(this.yAxisLeft.drawIcons=!0,this.yAxisRight.drawIcons=!0):(this.yAxisLeft.drawIcons=!1,this.yAxisRight.drawIcons=!1),this.yAxisRight.master=!s,this.yAxisRight.masterAxis=this.yAxisLeft,0==this.yAxisRight.master?(1==r?this.yAxisLeft.lineOffset=this.yAxisRight.width:this.yAxisLeft.lineOffset=0,n=this.yAxisLeft.redraw()||n,n=this.yAxisRight.redraw()||n):n=this.yAxisRight.redraw()||n;for(var p=["__barStackLeft","__barStackRight","__lineStackLeft","__lineStackRight"],c=0;ct?-1:1});for(var a=0;a=0&&t._redrawLabel(o-2,e.val,i,"vis-y-axis vis-major",t.props.majorCharHeight),t.master===!0&&(n?t._redrawLine(o,i,"vis-grid vis-horizontal vis-major",t.options.majorLinesOffset,t.props.majorLineWidth):t._redrawLine(o,i,"vis-grid vis-horizontal vis-minor",t.options.minorLinesOffset,t.props.minorLineWidth))});var d=0;void 0!==this.options[i].title&&void 0!==this.options[i].title.text&&(d=this.props.titleCharHeight);var l=this.options.icons===!0?Math.max(this.options.iconWidth,d)+this.options.labelOffsetX+15:d+this.options.labelOffsetX+15;return this.maxLabelSize>this.width-l&&this.options.visible===!0?(this.width=this.maxLabelSize+l,this.options.width=this.width+"px",s.cleanupElements(this.DOMelements.lines),s.cleanupElements(this.DOMelements.labels),this.redraw(),e=!0):this.maxLabelSizethis.minWidth?(this.width=Math.max(this.minWidth,this.maxLabelSize+l),this.options.width=this.width+"px",s.cleanupElements(this.DOMelements.lines),s.cleanupElements(this.DOMelements.labels),this.redraw(),e=!0):(s.cleanupElements(this.DOMelements.lines),s.cleanupElements(this.DOMelements.labels),e=!1),e},o.prototype.convertValue=function(t){return this.scale.convertValue(t)},o.prototype.screenToValue=function(t){return this.scale.screenToValue(t)},o.prototype._redrawLabel=function(t,e,i,o,n){var r=s.getDOMElement("div",this.DOMelements.labels,this.dom.frame);r.className=o,r.innerHTML=e,"left"===i?(r.style.left="-"+this.options.labelOffsetX+"px",r.style.textAlign="right"):(r.style.right="-"+this.options.labelOffsetX+"px",r.style.textAlign="left"),r.style.top=t-.5*n+this.options.labelOffsetY+"px",e+="";var a=Math.max(this.props.majorCharWidth,this.props.minorCharWidth);this.maxLabelSize.5*(h.magnitudefactor*h.minorSteps[h.minorStepIdx])?e+h.magnitudefactor*h.minorSteps[h.minorStepIdx]:e};i&&(this._start-=2*this.magnitudefactor*this.minorSteps[this.minorStepIdx],this._start=d(this._start)),o&&(this._end+=this.magnitudefactor*this.minorSteps[this.minorStepIdx],this._end=d(this._end)),this.determineScale()}}i.prototype.setCharHeight=function(t){this.majorCharHeight=t},i.prototype.setHeight=function(t){this.containerHeight=t},i.prototype.determineScale=function(){var t=this._end-this._start;this.scale=this.containerHeight/t;var e=this.majorCharHeight/this.scale,i=t>0?Math.round(Math.log(t)/Math.LN10):0;this.minorStepIdx=-1,this.magnitudefactor=Math.pow(10,i);var o=0;0>i&&(o=i);for(var n=!1,s=o;Math.abs(s)<=Math.abs(i);s++){this.magnitudefactor=Math.pow(10,s);for(var r=0;r=e){n=!0,this.minorStepIdx=r;break}}if(n===!0)break}},i.prototype.is_major=function(t){return t%(this.magnitudefactor*this.majorSteps[this.minorStepIdx])===0},i.prototype.getStep=function(){return this.magnitudefactor*this.minorSteps[this.minorStepIdx]},i.prototype.getFirstMajor=function(){var t=this.magnitudefactor*this.majorSteps[this.minorStepIdx];return this.convertValue(this._start+(t-this._start%t)%t)},i.prototype.formatValue=function(t){var e=t.toPrecision(5);return"function"==typeof this.formattingFunction&&(e=this.formattingFunction(t)),"number"==typeof e?""+e:"string"==typeof e?e:t.toPrecision(5)},i.prototype.getLines=function(){for(var t=[],e=this.getStep(),i=(e-this._start%e)%e,o=this._start+i;this._end-o>1e-5;o+=e)o!=this._start&&t.push({major:this.is_major(o),y:this.convertValue(o),val:this.formatValue(o)});return t},i.prototype.followScale=function(t){var e=this.minorStepIdx,i=this._start,o=this._end,n=this,s=function(){n.magnitudefactor*=2},r=function(){n.magnitudefactor/=2};t.minorStepIdx<=1&&this.minorStepIdx<=1||t.minorStepIdx>1&&this.minorStepIdx>1||(t.minorStepIdxo+1e-5)r(),d=!1;else{if(!this.autoScaleStart&&this._start=0)){r(),d=!1;continue}console.warn("Can't adhere to given 'min' range, due to zeroalign")}this.autoScaleStart&&this.autoScaleEnd&&o-i>c?(s(),d=!1):d=!0}}},i.prototype.convertValue=function(t){return this.containerHeight-(t-this._start)*this.scale},i.prototype.screenToValue=function(t){return(this.containerHeight-t)/this.scale+this._start},t.exports=i},function(t,e,i){function o(t,e,i,o){this.id=e;var n=["sampling","style","sort","yAxisOrientation","barChart","drawPoints","shaded","interpolation","zIndex","excludeFromStacking","excludeFromLegend"];this.options=s.selectiveBridgeObject(n,i),this.usingDefaultStyle=void 0===t.className,this.groupsUsingDefaultStyles=o,this.zeroPosition=0,this.update(t),1==this.usingDefaultStyle&&(this.groupsUsingDefaultStyles[0]+=1),this.itemsData=[],this.visible=void 0===t.visible?!0:t.visible}var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=i(1),r=(i(8),i(55)),a=i(57),h=i(56);o.prototype.setItems=function(t){null!=t?(this.itemsData=t,1==this.options.sort&&s.insertSort(this.itemsData,function(t,e){return t.x>e.x?1:-1})):this.itemsData=[]},o.prototype.getItems=function(){return this.itemsData},o.prototype.setZeroPosition=function(t){this.zeroPosition=t},o.prototype.setOptions=function(t){if(void 0!==t){var e=["sampling","style","sort","yAxisOrientation","barChart","zIndex","excludeFromStacking","excludeFromLegend"];s.selectiveDeepExtend(e,this.options,t),"function"==typeof t.drawPoints&&(t.drawPoints={onRender:t.drawPoints}),s.mergeOptions(this.options,t,"interpolation"),s.mergeOptions(this.options,t,"drawPoints"),s.mergeOptions(this.options,t,"shaded"),t.interpolation&&"object"==n(t.interpolation)&&t.interpolation.parametrization&&("uniform"==t.interpolation.parametrization?this.options.interpolation.alpha=0:"chordal"==t.interpolation.parametrization?this.options.interpolation.alpha=1:(this.options.interpolation.parametrization="centripetal",this.options.interpolation.alpha=.5))}},o.prototype.update=function(t){this.group=t,this.content=t.content||"graph",this.className=t.className||this.className||"vis-graph-group"+this.groupsUsingDefaultStyles[0]%10,this.visible=void 0===t.visible?!0:t.visible,this.style=t.style,this.setOptions(t.options)},o.prototype.getLegend=function(t,e,i,o,n){if(void 0==i||null==i){var s=document.createElementNS("http://www.w3.org/2000/svg","svg");i={svg:s,svgElements:{},options:this.options,groups:[this]}}switch(void 0!=o&&null!=o||(o=0),void 0!=n&&null!=n||(n=.5*e),this.options.style){case"line":a.drawIcon(this,o,n,t,e,i);break;case"points":case"point":h.drawIcon(this,o,n,t,e,i);break;case"bar":r.drawIcon(this,o,n,t,e,i)}return{icon:i.svg,label:this.content,orientation:this.options.yAxisOrientation}},o.prototype.getYRange=function(t){for(var e=t[0].y,i=t[0].y,o=0;ot[o].y?t[o].y:e,i=i0&&(i=Math.min(i,Math.abs(e[o-1].screen_x-e[o].screen_x))),0===i&&(void 0===t[e[o].screen_x]&&(t[e[o].screen_x]={amount:0,resolved:0,accumulatedPositive:0,accumulatedNegative:0}),t[e[o].screen_x].amount+=1)},o._getSafeDrawData=function(t,e,i){var o,n;return t0?(o=i>t?i:t,n=0,"left"===e.options.barChart.align?n-=.5*t:"right"===e.options.barChart.align&&(n+=.5*t)):(o=e.options.barChart.width,n=0,"left"===e.options.barChart.align?n-=.5*e.options.barChart.width:"right"===e.options.barChart.align&&(n+=.5*e.options.barChart.width)),{width:o,offset:n}},o.getStackedYRange=function(t,e,i,n,s){if(t.length>0){t.sort(function(t,e){return t.screen_x===e.screen_x?t.groupIde[s].screen_y?e[s].screen_y:o,n=nt[r].accumulatedNegative?t[r].accumulatedNegative:o,o=o>t[r].accumulatedPositive?t[r].accumulatedPositive:o,n=n0){var i=[];return i=1==e.options.interpolation.enabled?o._catmullRom(t,e):o._linear(t)}},o.drawIcon=function(t,e,i,o,s,r){var a,h,d=.5*s,l=n.getSVGElement("rect",r.svgElements,r.svg);if(l.setAttributeNS(null,"x",e),l.setAttributeNS(null,"y",i-d),l.setAttributeNS(null,"width",o),l.setAttributeNS(null,"height",2*d),l.setAttributeNS(null,"class","vis-outline"),a=n.getSVGElement("path",r.svgElements,r.svg),a.setAttributeNS(null,"class",t.className),void 0!==t.style&&a.setAttributeNS(null,"style",t.style),a.setAttributeNS(null,"d","M"+e+","+i+" L"+(e+o)+","+i),1==t.options.shaded.enabled&&(h=n.getSVGElement("path",r.svgElements,r.svg),"top"==t.options.shaded.orientation?h.setAttributeNS(null,"d","M"+e+", "+(i-d)+"L"+e+","+i+" L"+(e+o)+","+i+" L"+(e+o)+","+(i-d)):h.setAttributeNS(null,"d","M"+e+","+i+" L"+e+","+(i+d)+" L"+(e+o)+","+(i+d)+"L"+(e+o)+","+i),h.setAttributeNS(null,"class",t.className+" vis-icon-fill"),void 0!==t.options.shaded.style&&""!==t.options.shaded.style&&h.setAttributeNS(null,"style",t.options.shaded.style)),1==t.options.drawPoints.enabled){var c={style:t.options.drawPoints.style,styles:t.options.drawPoints.styles,size:t.options.drawPoints.size,className:t.className};n.drawPoint(e+.5*o,i,c,r.svgElements,r.svg)}},o.drawShading=function(t,e,i,o){if(1==e.options.shaded.enabled){var s=Number(o.svg.style.height.replace("px","")),r=n.getSVGElement("path",o.svgElements,o.svg),a="L";1==e.options.interpolation.enabled&&(a="C");var h,d=0;d="top"==e.options.shaded.orientation?0:"bottom"==e.options.shaded.orientation?s:Math.min(Math.max(0,e.zeroPosition),s),h="group"==e.options.shaded.orientation&&null!=i&&void 0!=i?"M"+t[0][0]+","+t[0][1]+" "+this.serializePath(t,a,!1)+" L"+i[i.length-1][0]+","+i[i.length-1][1]+" "+this.serializePath(i,a,!0)+i[0][0]+","+i[0][1]+" Z":"M"+t[0][0]+","+t[0][1]+" "+this.serializePath(t,a,!1)+" V"+d+" H"+t[0][0]+" Z",r.setAttributeNS(null,"class",e.className+" vis-fill"),void 0!==e.options.shaded.style&&r.setAttributeNS(null,"style",e.options.shaded.style),r.setAttributeNS(null,"d",h)}},o.draw=function(t,e,i){if(null!=t&&void 0!=t){var o=n.getSVGElement("path",i.svgElements,i.svg);o.setAttributeNS(null,"class",e.className),void 0!==e.style&&o.setAttributeNS(null,"style",e.style);var s="L";1==e.options.interpolation.enabled&&(s="C"),o.setAttributeNS(null,"d","M"+t[0][0]+","+t[0][1]+" "+this.serializePath(t,s,!1))}},o.serializePath=function(t,e,i){if(t.length<2)return"";var o=e;if(i)for(var n=t.length-2;n>0;n--)o+=t[n][0]+","+t[n][1]+" ";else for(var n=1;nl;l++)e=0==l?t[0]:t[l-1],i=t[l],o=t[l+1],n=d>l+2?t[l+2]:o,s={screen_x:(-e.screen_x+6*i.screen_x+o.screen_x)*h,screen_y:(-e.screen_y+6*i.screen_y+o.screen_y)*h},r={screen_x:(i.screen_x+6*o.screen_x-n.screen_x)*h,screen_y:(i.screen_y+6*o.screen_y-n.screen_y)*h},a.push([s.screen_x,s.screen_y]),a.push([r.screen_x,r.screen_y]),a.push([o.screen_x,o.screen_y]);return a},o._catmullRom=function(t,e){var i=e.options.interpolation.alpha;if(0==i||void 0===i)return this._catmullRomUniform(t);var o,n,s,r,a,h,d,l,c,u,p,f,m,v,g,y,b,w,_,x=[];x.push([Math.round(t[0].screen_x),Math.round(t[0].screen_y)]);for(var k=t.length,O=0;k-1>O;O++)o=0==O?t[0]:t[O-1],n=t[O],s=t[O+1],r=k>O+2?t[O+2]:s,d=Math.sqrt(Math.pow(o.screen_x-n.screen_x,2)+Math.pow(o.screen_y-n.screen_y,2)),l=Math.sqrt(Math.pow(n.screen_x-s.screen_x,2)+Math.pow(n.screen_y-s.screen_y,2)),c=Math.sqrt(Math.pow(s.screen_x-r.screen_x,2)+Math.pow(s.screen_y-r.screen_y,2)),v=Math.pow(c,i),y=Math.pow(c,2*i),g=Math.pow(l,i),b=Math.pow(l,2*i),_=Math.pow(d,i),w=Math.pow(d,2*i),u=2*w+3*_*g+b,p=2*y+3*v*g+b,f=3*_*(_+g),f>0&&(f=1/f),m=3*v*(v+g),m>0&&(m=1/m),a={screen_x:(-b*o.screen_x+u*n.screen_x+w*s.screen_x)*f,screen_y:(-b*o.screen_y+u*n.screen_y+w*s.screen_y)*f},h={screen_x:(y*n.screen_x+p*s.screen_x-b*r.screen_x)*m,screen_y:(y*n.screen_y+p*s.screen_y-b*r.screen_y)*m},0==a.screen_x&&0==a.screen_y&&(a=n),0==h.screen_x&&0==h.screen_y&&(h=s),x.push([a.screen_x,a.screen_y]),x.push([h.screen_x,h.screen_y]),x.push([s.screen_x,s.screen_y]);return x},o._linear=function(t){for(var e=[],i=0;it?-1:1});for(var i=0;i")}this.dom.textArea.innerHTML=s,this.dom.textArea.style.lineHeight=.75*this.options.iconSize+this.options.iconSpacing+"px"}},o.prototype.drawLegendIcons=function(){if(this.dom.frame.parentNode){var t=Object.keys(this.groups);t.sort(function(t,e){return e>t?-1:1}),s.resetElements(this.svgElements);var e=window.getComputedStyle(this.dom.frame).paddingTop,i=Number(e.replace("px","")),o=i,n=this.options.iconSize,r=.75*this.options.iconSize,a=i+.5*r+3;this.svg.style.width=n+5+i+"px";for(var h=0;h0){var i=this.groupIndex%this.groupsArray.length;this.groupIndex++,e={},e.color=this.groups[this.groupsArray[i]],this.groups[t]=e}else{var o=this.defaultIndex%this.defaultGroups.length;this.defaultIndex++,e={},e.color=this.defaultGroups[o],this.groups[t]=e}return e}},{key:"add",value:function(t,e){return this.groups[t]=e,this.groupsArray.push(t),e}}]),t}();e["default"]=r},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s=function(){function t(t,e){for(var i=0;it.left&&this.shape.topt.top}},{key:"isBoundingBoxOverlappingWith",value:function(t){return this.shape.boundingBox.leftt.left&&this.shape.boundingBox.topt.top}}],[{key:"parseOptions",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!1:arguments[2],o=arguments.length<=3||void 0===arguments[3]?{}:arguments[3],n=["color","font","fixed","shadow"];if(A.selectiveNotDeepExtend(n,t,e,i),A.mergeOptions(t,e,"shadow",i,o),void 0!==e.color&&null!==e.color){var s=A.parseColor(e.color);A.fillIfDefined(t.color,s)}else i===!0&&null===e.color&&(t.color=A.bridgeObject(o.color));void 0!==e.fixed&&null!==e.fixed&&("boolean"==typeof e.fixed?(t.fixed.x=e.fixed,t.fixed.y=e.fixed):(void 0!==e.fixed.x&&"boolean"==typeof e.fixed.x&&(t.fixed.x=e.fixed.x),void 0!==e.fixed.y&&"boolean"==typeof e.fixed.y&&(t.fixed.y=e.fixed.y))),void 0!==e.font&&null!==e.font?a["default"].parseOptions(t.font,e):i===!0&&null===e.font&&(t.font=A.bridgeObject(o.font)),void 0!==e.scaling&&A.mergeOptions(t.scaling,e.scaling,"label",i,o.scaling)}}]),t}();e["default"]=B},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n=function(){function t(t,e){var i=[],o=!0,n=!1,s=void 0;try{for(var r,a=t[Symbol.iterator]();!(o=(r=a.next()).done)&&(i.push(r.value),!e||i.length!==e);o=!0);}catch(h){n=!0,s=h}finally{try{!o&&a["return"]&&a["return"]()}finally{if(n)throw s}}return i}return function(e,i){if(Array.isArray(e))return e;if(Symbol.iterator in Object(e))return t(e,i);throw new TypeError("Invalid attempt to destructure non-iterable instance")}}(),s="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},r=function(){function t(t,e){for(var i=0;i=this.nodeOptions.scaling.label.maxVisible&&(r=Number(this.nodeOptions.scaling.label.maxVisible)/this.body.view.scale);var h=this.size.yLine,d=this._getColor(a),l=n(d,2),c=l[0],u=l[1],p=this._setAlignment(t,i,h,s),f=n(p,2);i=f[0],h=f[1],t.font=(e&&this.nodeOptions.labelHighlightBold?"bold ":"")+r+"px "+this.fontOptions.face,t.fillStyle=c,this.isEdgeLabel||"left"!==this.fontOptions.align?t.textAlign="center":(t.textAlign=this.fontOptions.align,i-=.5*this.size.width),this.fontOptions.strokeWidth>0&&(t.lineWidth=this.fontOptions.strokeWidth,t.strokeStyle=u,t.lineJoin="round");for(var m=0;m0&&t.strokeText(this.lines[m],i,h),t.fillText(this.lines[m],i,h),h+=r}},{key:"_setAlignment",value:function(t,e,i,o){if(this.isEdgeLabel&&"horizontal"!==this.fontOptions.align&&this.pointToSelf===!1){e=0,i=0;var n=2;"top"===this.fontOptions.align?(t.textBaseline="alphabetic",i-=2*n):"bottom"===this.fontOptions.align?(t.textBaseline="hanging",i+=2*n):t.textBaseline="middle"}else t.textBaseline=o;return[e,i]}},{key:"_getColor",value:function(t){var e=this.fontOptions.color||"#000000",i=this.fontOptions.strokeColor||"#ffffff";if(t<=this.nodeOptions.scaling.label.drawThreshold){var o=Math.max(0,Math.min(1,1-(this.nodeOptions.scaling.label.drawThreshold-t)));e=a.overrideOpacity(e,o),i=a.overrideOpacity(i,o)}return[e,i]}},{key:"getTextSize",value:function(t){var e=arguments.length<=1||void 0===arguments[1]?!1:arguments[1],i={width:this._processLabel(t,e),height:this.fontOptions.size*this.lineCount,lineCount:this.lineCount};return i}},{key:"calculateLabelSize",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?0:arguments[2],o=arguments.length<=3||void 0===arguments[3]?0:arguments[3],n=arguments.length<=4||void 0===arguments[4]?"middle":arguments[4];this.labelDirty===!0&&(this.size.width=this._processLabel(t,e)),this.size.height=this.fontOptions.size*this.lineCount,this.size.left=i-.5*this.size.width,this.size.top=o-.5*this.size.height,this.size.yLine=o+.5*(1-this.lineCount)*this.fontOptions.size,"hanging"===n&&(this.size.top+=.5*this.fontOptions.size,this.size.top+=4,this.size.yLine+=4),this.labelDirty=!1}},{key:"_processLabel",value:function(t,e){var i=0,o=[""],n=0;if(void 0!==this.nodeOptions.label){o=String(this.nodeOptions.label).split("\n"),n=o.length,t.font=(e&&this.nodeOptions.labelHighlightBold?"bold ":"")+this.fontOptions.size+"px "+this.fontOptions.face,i=t.measureText(o[0]).width;for(var s=1;n>s;s++){var r=t.measureText(o[s]).width;i=r>i?r:i}}return this.lines=o,this.lineCount=n,i}}],[{key:"parseOptions",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!1:arguments[2];if("string"==typeof e.font){var o=e.font.split(" ");t.size=o[0].replace("px",""),t.face=o[1],t.color=o[2]}else"object"===s(e.font)&&a.fillIfDefined(t,e.font,i);t.size=Number(t.size)}}]),t}();e["default"]=h},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore(),this.updateBoundingBox(e,i,t,o),this.labelModule.draw(t,e,i,o)}},{key:"updateBoundingBox",value:function(t,e,i,o){this.resize(i,o),this.left=t-.5*this.width,this.top=e-.5*this.height;var n=this.options.shapeProperties.borderRadius;this.boundingBox.left=this.left-n,this.boundingBox.top=this.top-n,this.boundingBox.bottom=this.top+this.height+n,this.boundingBox.right=this.left+this.width+n}},{key:"distanceToBorder",value:function(t,e){this.resize(t);var i=this.options.borderWidth;return Math.min(Math.abs(this.width/2/Math.cos(e)),Math.abs(this.height/2/Math.sin(e)))+i}}]),e}(d["default"]);e["default"]=l},function(t,e){function i(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var o=function(){function t(t,e){for(var i=0;ithis.imageObj.height?(o=this.imageObj.width/this.imageObj.height,e=2*this.options.size*o||this.imageObj.width,i=2*this.options.size||this.imageObj.height):(o=this.imageObj.width&&this.imageObj.height?this.imageObj.height/this.imageObj.width:1,e=2*this.options.size,i=2*this.options.size*o):(e=this.imageObj.width,i=this.imageObj.height),this.width=e,this.height=i,this.radius=.5*this.width}}},{key:"_drawRawCircle",value:function(t,e,i,o,n,s){var r=this.options.borderWidth,a=this.options.borderWidthSelected||2*this.options.borderWidth,h=(o?a:r)/this.body.view.scale;t.lineWidth=Math.min(this.width,h),t.strokeStyle=o?this.options.color.highlight.border:n?this.options.color.hover.border:this.options.color.border,t.fillStyle=o?this.options.color.highlight.background:n?this.options.color.hover.background:this.options.color.background,t.circle(e,i,s),this.enableShadow(t),t.fill(),this.disableShadow(t),t.save(),h>0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore()}},{key:"_drawImageAtPosition",value:function(t){if(0!=this.imageObj.width){t.globalAlpha=1,this.enableShadow(t);var e=this.imageObj.width/this.width/this.body.view.scale;if(e>2&&this.options.shapeProperties.interpolation===!0){var i=this.imageObj.width,o=this.imageObj.height,n=document.createElement("canvas");n.width=i,n.height=i;var s=n.getContext("2d");e*=.5,i*=.5,o*=.5,s.drawImage(this.imageObj,0,0,i,o);for(var r=0,a=1;e>2&&4>a;)s.drawImage(n,r,0,i,o,r+i,0,i/2,o/2),r+=i,e*=.5,i*=.5,o*=.5,a+=1;t.drawImage(n,r,0,i,o,this.left,this.top,this.width,this.height)}else t.drawImage(this.imageObj,this.left,this.top,this.width,this.height);this.disableShadow(t)}}},{key:"_drawImageLabel",value:function(t,e,i,o){var n,s=0;if(void 0!==this.height){s=.5*this.height;var r=this.labelModule.getTextSize(t);r.lineCount>=1&&(s+=r.height/2)}n=i+s,this.options.label&&(this.labelOffset=s),this.labelModule.draw(t,e,n,o,"hanging")}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore(),this.updateBoundingBox(e,i,t,o),this.labelModule.draw(t,e,i,o)}},{key:"updateBoundingBox",value:function(t,e,i,o){this.resize(i,o),this.left=t-.5*this.width,this.top=e-.5*this.height,this.boundingBox.left=this.left,this.boundingBox.top=this.top,this.boundingBox.bottom=this.top+this.height,this.boundingBox.right=this.left+this.width}},{key:"distanceToBorder",value:function(t,e){return this._distanceToBorder(t,e)}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore(),void 0!==this.options.label){var l=n+.5*this.height+3;this.labelModule.draw(t,o,l,s,"hanging")}this.updateBoundingBox(o,n)}},{key:"updateBoundingBox",value:function(t,e){this.boundingBox.top=e-this.options.size,this.boundingBox.left=t-this.options.size,this.boundingBox.right=t+this.options.size,this.boundingBox.bottom=e+this.options.size,void 0!==this.options.label&&this.labelModule.size.width>0&&(this.boundingBox.left=Math.min(this.boundingBox.left,this.labelModule.size.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelModule.size.left+this.labelModule.size.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelModule.size.height+3))}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore(),this.updateBoundingBox(e,i,t,o),this.labelModule.draw(t,e,i,o)}},{key:"updateBoundingBox",value:function(t,e,i,o){this.resize(i,o),this.left=t-.5*this.width,this.top=e-.5*this.height,this.boundingBox.left=this.left,this.boundingBox.top=this.top,this.boundingBox.bottom=this.top+this.height,this.boundingBox.right=this.left+this.width}},{key:"distanceToBorder",value:function(t,e){this.resize(t);var i=.5*this.width,o=.5*this.height,n=Math.sin(e)*i,s=Math.cos(e)*o;return i*o/Math.sqrt(n*n+s*s)}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0){var i=5;this.boundingBox.left=Math.min(this.boundingBox.left,this.labelModule.size.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelModule.size.left+this.labelModule.size.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelModule.size.height+i)}}},{key:"_icon",value:function(t,e,i,o){var n=Number(this.options.icon.size);void 0!==this.options.icon.code?(t.font=(o?"bold ":"")+n+"px "+this.options.icon.face,t.fillStyle=this.options.icon.color||"black",t.textAlign="center",t.textBaseline="middle",this.enableShadow(t),t.fillText(this.options.icon.code,e,i),this.disableShadow(t)):console.error("When using the icon shape, you need to define the code in the icon options object. This can be done per node or globally.")}},{key:"distanceToBorder",value:function(t,e){return this._distanceToBorder(t,e)}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore(),t.closePath()}this._drawImageAtPosition(t),this._drawImageLabel(t,e,i,o||n),this.updateBoundingBox(e,i)}},{key:"updateBoundingBox",value:function(t,e){this.resize(),this.left=t-this.width/2,this.top=e-this.height/2,this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height,void 0!==this.options.label&&this.labelModule.size.width>0&&(this.boundingBox.left=Math.min(this.boundingBox.left,this.labelModule.size.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelModule.size.left+this.labelModule.size.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelOffset))}},{key:"distanceToBorder", +value:function(t,e){return this._distanceToBorder(t,e)}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;ii.shape.height?(r=i.x+.5*i.shape.width,a=i.y-h):(r=i.x+h,a=i.y-.5*i.shape.height),s=this._pointOnCircle(r,a,h,.125),this.labelModule.draw(t,s.x,s.y,n)}}}},{key:"isOverlappingWith",value:function(t){if(this.connected){var e=10,i=this.from.x,o=this.from.y,n=this.to.x,s=this.to.y,r=t.left,a=t.top,h=this.edgeType.getDistanceToEdge(i,o,n,s,r,a);return e>h}return!1}},{key:"_rotateForLabelAlignment",value:function(t){var e=this.from.y-this.to.y,i=this.from.x-this.to.x,o=Math.atan2(e,i);(-1>o&&0>i||o>0&&0>i)&&(o+=Math.PI),t.rotate(o)}},{key:"_pointOnCircle",value:function(t,e,i,o){var n=2*o*Math.PI;return{x:t+i*Math.cos(n),y:e-i*Math.sin(n)}}},{key:"select",value:function(){this.selected=!0}},{key:"unselect",value:function(){this.selected=!1}},{key:"cleanup",value:function(){return this.edgeType.cleanup()}}],[{key:"parseOptions",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!1:arguments[2],o=arguments.length<=3||void 0===arguments[3]?{}:arguments[3],n=["arrowStrikethrough","id","from","hidden","hoverWidth","label","labelHighlightBold","length","line","opacity","physics","scaling","selectionWidth","selfReferenceSize","to","title","value","width"];if(g.selectiveDeepExtend(n,t,e,i),g.mergeOptions(t,e,"smooth",i,o),g.mergeOptions(t,e,"shadow",i,o),void 0!==e.dashes&&null!==e.dashes?t.dashes=e.dashes:i===!0&&null===e.dashes&&(t.dashes=Object.create(o.dashes)),void 0!==e.scaling&&null!==e.scaling?(void 0!==e.scaling.min&&(t.scaling.min=e.scaling.min),void 0!==e.scaling.max&&(t.scaling.max=e.scaling.max),g.mergeOptions(t.scaling,e.scaling,"label",i,o.scaling)):i===!0&&null===e.scaling&&(t.scaling=Object.create(o.scaling)),void 0!==e.arrows&&null!==e.arrows)if("string"==typeof e.arrows){var r=e.arrows.toLowerCase();t.arrows.to.enabled=-1!=r.indexOf("to"),t.arrows.middle.enabled=-1!=r.indexOf("middle"),t.arrows.from.enabled=-1!=r.indexOf("from")}else{if("object"!==s(e.arrows))throw new Error("The arrow newOptions can only be an object or a string. Refer to the documentation. You used:"+JSON.stringify(e.arrows));g.mergeOptions(t.arrows,e.arrows,"to",i,o.arrows),g.mergeOptions(t.arrows,e.arrows,"middle",i,o.arrows),g.mergeOptions(t.arrows,e.arrows,"from",i,o.arrows)}else i===!0&&null===e.arrows&&(t.arrows=Object.create(o.arrows));if(void 0!==e.color&&null!==e.color)if(t.color=g.deepExtend({},t.color,!0),g.isString(e.color))t.color.color=e.color,t.color.highlight=e.color,t.color.hover=e.color,t.color.inherit=!1;else{var a=!1;void 0!==e.color.color&&(t.color.color=e.color.color,a=!0),void 0!==e.color.highlight&&(t.color.highlight=e.color.highlight,a=!0),void 0!==e.color.hover&&(t.color.hover=e.color.hover,a=!0),void 0!==e.color.inherit&&(t.color.inherit=e.color.inherit),void 0!==e.color.opacity&&(t.color.opacity=Math.min(1,Math.max(0,e.color.opacity))),void 0===e.color.inherit&&a===!0&&(t.color.inherit=!1)}else i===!0&&null===e.color&&(t.color=g.bridgeObject(o.color));void 0!==e.font&&null!==e.font?h["default"].parseOptions(t.font,e):i===!0&&null===e.font&&(t.font=g.bridgeObject(o.font))}}]),t}();e["default"]=y},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){var i=[],o=!0,n=!1,s=void 0;try{for(var r,a=t[Symbol.iterator]();!(o=(r=a.next()).done)&&(i.push(r.value),!e||i.length!==e);o=!0);}catch(h){n=!0,s=h}finally{try{!o&&a["return"]&&a["return"]()}finally{if(n)throw s}}return i}return function(e,i){if(Array.isArray(e))return e;if(Symbol.iterator in Object(e))return t(e,i);throw new TypeError("Invalid attempt to destructure non-iterable instance")}}(),h=function(){function t(t,e){for(var i=0;iMath.abs(e)||this.options.smooth.forceDirection===!0||"horizontal"===this.options.smooth.forceDirection)&&"vertical"!==this.options.smooth.forceDirection?(o=this.from.y,s=this.to.y,i=this.from.x-r*t,n=this.to.x+r*t):(o=this.from.y-r*e,s=this.to.y+r*e,i=this.from.x,n=this.to.x),[{x:i,y:o},{x:n,y:s}]}},{key:"getViaNode",value:function(){return this._getViaCoordinates()}},{key:"_findBorderPosition",value:function(t,e){return this._findBorderPositionBezier(t,e)}},{key:"_getDistanceToEdge",value:function(t,e,i,o,n,s){var r=arguments.length<=6||void 0===arguments[6]?this._getViaCoordinates():arguments[6],h=a(r,2),d=h[0],l=h[1];return this._getDistanceToBezierEdge(t,e,i,o,n,s,d,l)}},{key:"getPoint",value:function(t){var e=arguments.length<=1||void 0===arguments[1]?this._getViaCoordinates():arguments[1],i=a(e,2),o=i[0],n=i[1],s=t,r=[];r[0]=Math.pow(1-s,3),r[1]=3*s*Math.pow(1-s,2),r[2]=3*Math.pow(s,2)*(1-s),r[3]=Math.pow(s,3);var h=r[0]*this.fromPoint.x+r[1]*o.x+r[2]*n.x+r[3]*this.toPoint.x,d=r[0]*this.fromPoint.y+r[1]*o.y+r[2]*n.y+r[3]*this.toPoint.y;return{x:h,y:d}}}]),e}(l["default"]);e["default"]=c},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;il;l++)c=.1*l,v[0]=Math.pow(1-c,3),v[1]=3*c*Math.pow(1-c,2),v[2]=3*Math.pow(c,2)*(1-c),v[3]=Math.pow(c,3),u=v[0]*t+v[1]*r.x+v[2]*a.x+v[3]*i,p=v[0]*e+v[1]*r.y+v[2]*a.y+v[3]*o,l>0&&(d=this._getDistanceToLine(f,m,u,p,n,s),h=h>d?d:h),f=u,m=p;return h}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i=l&&h>d;){var m=.5*(l+c);if(i=this.getPoint(m,a),o=Math.atan2(p.y-i.y,p.x-i.x),n=p.distanceToBorder(e,o),s=Math.sqrt(Math.pow(i.x-p.x,2)+Math.pow(i.y-p.y,2)),r=n-s,Math.abs(r)r?f===!1?l=m:c=m:f===!1?c=m:l=m,d++}return i.t=m,i}},{key:"_getDistanceToBezierEdge",value:function(t,e,i,o,n,s,r){var a=1e9,h=void 0,d=void 0,l=void 0,c=void 0,u=void 0,p=t,f=e;for(d=1;10>d;d++)l=.1*d,c=Math.pow(1-l,2)*t+2*l*(1-l)*r.x+Math.pow(l,2)*i,u=Math.pow(1-l,2)*e+2*l*(1-l)*r.y+Math.pow(l,2)*o,d>0&&(h=this._getDistanceToLine(p,f,c,u,n,s),a=a>h?h:a),p=c,f=u;return a}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n=function(){function t(t,e){var i=[],o=!0,n=!1,s=void 0;try{for(var r,a=t[Symbol.iterator]();!(o=(r=a.next()).done)&&(i.push(r.value),!e||i.length!==e);o=!0);}catch(h){n=!0,s=h}finally{try{!o&&a["return"]&&a["return"]()}finally{if(n)throw s}}return i}return function(e,i){if(Array.isArray(e))return e;if(Symbol.iterator in Object(e))return t(e,i);throw new TypeError("Invalid attempt to destructure non-iterable instance")}}(),s=function(){function t(t,e){for(var i=0;io.shape.height?(e=o.x+.5*o.shape.width,i=o.y-n):(e=o.x+n,i=o.y-.5*o.shape.height),[e,i,n]}},{key:"_pointOnCircle",value:function(t,e,i,o){var n=2*o*Math.PI;return{x:t+i*Math.cos(n),y:e-i*Math.sin(n)}}},{key:"_findBorderPositionCircle",value:function(t,e,i){for(var o=i.x,n=i.y,s=i.low,r=i.high,a=i.direction,h=10,d=0,l=this.options.selfReferenceSize,c=void 0,u=void 0,p=void 0,f=void 0,m=void 0,v=.05,g=.5*(s+r);r>=s&&h>d&&(g=.5*(s+r),c=this._pointOnCircle(o,n,l,g),u=Math.atan2(t.y-c.y,t.x-c.x),p=t.distanceToBorder(e,u),f=Math.sqrt(Math.pow(c.x-t.x,2)+Math.pow(c.y-t.y,2)), +m=p-f,!(Math.abs(m)0?a>0?s=g:r=g:a>0?r=g:s=g,d++;return c.t=g,c}},{key:"getLineWidth",value:function(t,e){return t===!0?Math.max(this.selectionWidth,.3/this.body.view.scale):e===!0?Math.max(this.hoverWidth,.3/this.body.view.scale):Math.max(this.options.width,.3/this.body.view.scale)}},{key:"getColor",value:function(t,e,i){var o=this.options.color;if(o.inherit!==!1){if("both"===o.inherit&&this.from.id!==this.to.id){var n=t.createLinearGradient(this.from.x,this.from.y,this.to.x,this.to.y),s=void 0,a=void 0;return s=this.from.options.color.highlight.border,a=this.to.options.color.highlight.border,this.from.selected===!1&&this.to.selected===!1?(s=r.overrideOpacity(this.from.options.color.border,this.options.color.opacity),a=r.overrideOpacity(this.to.options.color.border,this.options.color.opacity)):this.from.selected===!0&&this.to.selected===!1?a=this.to.options.color.border:this.from.selected===!1&&this.to.selected===!0&&(s=this.from.options.color.border),n.addColorStop(0,s),n.addColorStop(1,a),n}this.colorDirty===!0&&("to"===o.inherit?(this.color.highlight=this.to.options.color.highlight.border,this.color.hover=this.to.options.color.hover.border,this.color.color=r.overrideOpacity(this.to.options.color.border,o.opacity)):(this.color.highlight=this.from.options.color.highlight.border,this.color.hover=this.from.options.color.hover.border,this.color.color=r.overrideOpacity(this.from.options.color.border,o.opacity)))}else this.colorDirty===!0&&(this.color.highlight=o.highlight,this.color.hover=o.hover,this.color.color=r.overrideOpacity(o.color,o.opacity));return this.colorDirty=!1,e===!0?this.color.highlight:i===!0?this.color.hover:this.color.color}},{key:"_circle",value:function(t,e,i,o){this.enableShadow(t),t.beginPath(),t.arc(e,i,o,0,2*Math.PI,!1),t.stroke(),this.disableShadow(t)}},{key:"getDistanceToEdge",value:function(t,e,i,o,s,r,a){var h=0;if(this.from!=this.to)h=this._getDistanceToEdge(t,e,i,o,s,r,a);else{var d=this._getCircleData(),l=n(d,3),c=l[0],u=l[1],p=l[2],f=c-s,m=u-r;h=Math.abs(Math.sqrt(f*f+m*m)-p)}return this.labelModule.size.lefts&&this.labelModule.size.topr?0:h}},{key:"_getDistanceToLine",value:function(t,e,i,o,n,s){var r=i-t,a=o-e,h=r*r+a*a,d=((n-t)*r+(s-e)*a)/h;d>1?d=1:0>d&&(d=0);var l=t+d*r,c=e+d*a,u=l-n,p=c-s;return Math.sqrt(u*u+p*p)}},{key:"getArrowData",value:function(t,e,i,o,s){var r=void 0,a=void 0,h=void 0,d=void 0,l=void 0,c=void 0,u=this.getLineWidth(o,s);if("from"===e?(h=this.from,d=this.to,l=.1,c=this.options.arrows.from.scaleFactor):"to"===e?(h=this.to,d=this.from,l=-.1,c=this.options.arrows.to.scaleFactor):(h=this.to,d=this.from,c=this.options.arrows.middle.scaleFactor),h!=d)if("middle"!==e)if(this.options.smooth.enabled===!0){a=this.findBorderPosition(h,t,{via:i});var p=this.getPoint(Math.max(0,Math.min(1,a.t+l)),i);r=Math.atan2(a.y-p.y,a.x-p.x)}else r=Math.atan2(h.y-d.y,h.x-d.x),a=this.findBorderPosition(h,t);else r=Math.atan2(h.y-d.y,h.x-d.x),a=this.getPoint(.5,i);else{var f=this._getCircleData(t),m=n(f,3),v=m[0],g=m[1],y=m[2];"from"===e?(a=this.findBorderPosition(this.from,t,{x:v,y:g,low:.25,high:.6,direction:-1}),r=-2*a.t*Math.PI+1.5*Math.PI+.1*Math.PI):"to"===e?(a=this.findBorderPosition(this.from,t,{x:v,y:g,low:.6,high:1,direction:1}),r=-2*a.t*Math.PI+1.5*Math.PI-1.1*Math.PI):(a=this._pointOnCircle(v,g,y,.175),r=3.9269908169872414)}var b=15*c+3*u,w=a.x-.9*b*Math.cos(r),_=a.y-.9*b*Math.sin(r),x={x:w,y:_};return{point:a,core:x,angle:r,length:b}}},{key:"drawArrowHead",value:function(t,e,i,o){t.strokeStyle=this.getColor(t,e,i),t.fillStyle=t.strokeStyle,t.lineWidth=this.getLineWidth(e,i),t.arrow(o.point.x,o.point.y,o.angle,o.length),this.enableShadow(t),t.fill(),this.disableShadow(t)}},{key:"enableShadow",value:function(t){this.options.shadow.enabled===!0&&(t.shadowColor=this.options.shadow.color,t.shadowBlur=this.options.shadow.size,t.shadowOffsetX=this.options.shadow.x,t.shadowOffsetY=this.options.shadow.y)}},{key:"disableShadow",value:function(t){this.options.shadow.enabled===!0&&(t.shadowColor="rgba(0,0,0,0)",t.shadowBlur=0,t.shadowOffsetX=0,t.shadowOffsetY=0)}}]),t}();e["default"]=a},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i=this.to.y?this.from.x<=this.to.x?(t=this.from.x+i*s,e=this.from.y-i*s):this.from.x>this.to.x&&(t=this.from.x-i*s,e=this.from.y-i*s):this.from.ythis.to.x&&(t=this.from.x-i*s,e=this.from.y+i*s)),"discrete"===o&&(t=i*s>n?this.from.x:t)):Math.abs(this.from.x-this.to.x)>Math.abs(this.from.y-this.to.y)&&(this.from.y>=this.to.y?this.from.x<=this.to.x?(t=this.from.x+i*n,e=this.from.y-i*n):this.from.x>this.to.x&&(t=this.from.x-i*n,e=this.from.y-i*n):this.from.ythis.to.x&&(t=this.from.x-i*n,e=this.from.y+i*n)),"discrete"===o&&(e=i*n>s?this.from.y:e));else if("straightCross"===o)Math.abs(this.from.x-this.to.x)<=Math.abs(this.from.y-this.to.y)?(t=this.from.x,e=this.from.yMath.abs(this.from.y-this.to.y)&&(t=this.from.x=this.to.y?this.from.x<=this.to.x?(t=this.from.x+i*s,e=this.from.y-i*s,t=this.to.xthis.to.x&&(t=this.from.x-i*s,e=this.from.y-i*s,t=this.to.x>t?this.to.x:t):this.from.ythis.to.x&&(t=this.from.x-i*s,e=this.from.y+i*s,t=this.to.x>t?this.to.x:t)):Math.abs(this.from.x-this.to.x)>Math.abs(this.from.y-this.to.y)&&(this.from.y>=this.to.y?this.from.x<=this.to.x?(t=this.from.x+i*n,e=this.from.y-i*n,e=this.to.y>e?this.to.y:e):this.from.x>this.to.x&&(t=this.from.x-i*n,e=this.from.y-i*n,e=this.to.y>e?this.to.y:e):this.from.ythis.to.x&&(t=this.from.x-i*n,e=this.from.y+i*n,e=this.to.y1||this.startedStabilization===!0)&&setTimeout(function(){t.body.emitter.emit("stabilized",{iterations:e}),t.startedStabilization=!1,t.stabilizationIterations=0},0)}},{key:"physicsTick",value:function(){if(this.startedStabilization===!1&&(this.body.emitter.emit("startStabilizing"),this.startedStabilization=!0),this.stabilized===!1){if(this.adaptiveTimestep===!0&&this.adaptiveTimestepEnabled===!0){var t=1.2;this.adaptiveCounter%this.adaptiveInterval===0?(this.timestep=2*this.timestep,this.calculateForces(),this.moveNodes(),this.revert(),this.timestep=.5*this.timestep,this.calculateForces(),this.moveNodes(),this.calculateForces(),this.moveNodes(),this._evaluateStepQuality()===!0?this.timestep=t*this.timestep:this.timestep/ts))return!1;return!0}},{key:"moveNodes",value:function(){for(var t=this.physicsBody.physicsNodeIndices,e=this.options.maxVelocity?this.options.maxVelocity:1e9,i=0,o=0,n=5,s=0;se?s[t].x>0?e:-e:s[t].x,i.x+=s[t].x*o}else n[t].x=0,s[t].x=0;if(i.options.fixed.y===!1){var h=this.modelOptions.damping*s[t].y,d=(n[t].y-h)/i.options.mass;s[t].y+=d*o,s[t].y=Math.abs(s[t].y)>e?s[t].y>0?e:-e:s[t].y,i.y+=s[t].y*o}else n[t].y=0,s[t].y=0;var l=Math.sqrt(Math.pow(s[t].x,2)+Math.pow(s[t].y,2));return l}},{key:"calculateForces",value:function(){this.gravitySolver.solve(),this.nodesSolver.solve(),this.edgesSolver.solve()}},{key:"_freezeNodes",value:function(){var t=this.body.nodes;for(var e in t)t.hasOwnProperty(e)&&t[e].x&&t[e].y&&(this.freezeCache[e]={x:t[e].options.fixed.x,y:t[e].options.fixed.y},t[e].options.fixed.x=!0,t[e].options.fixed.y=!0)}},{key:"_restoreFrozenNodes",value:function(){var t=this.body.nodes;for(var e in t)t.hasOwnProperty(e)&&void 0!==this.freezeCache[e]&&(t[e].options.fixed.x=this.freezeCache[e].x,t[e].options.fixed.y=this.freezeCache[e].y);this.freezeCache={}}},{key:"stabilize",value:function(){var t=this,e=arguments.length<=0||void 0===arguments[0]?this.options.stabilization.iterations:arguments[0];return"number"!=typeof e&&(console.log("The stabilize method needs a numeric amount of iterations. Switching to default: ",this.options.stabilization.iterations),e=this.options.stabilization.iterations),0===this.physicsBody.physicsNodeIndices.length?void(this.ready=!0):(this.adaptiveTimestep=this.options.adaptiveTimestep,this.body.emitter.emit("_resizeNodes"),this.stopSimulation(),this.stabilized=!1,this.body.emitter.emit("_blockRedraw"),this.targetIterations=e,this.options.stabilization.onlyDynamicEdges===!0&&this._freezeNodes(),this.stabilizationIterations=0,void setTimeout(function(){return t._stabilizationBatch()},0))}},{key:"_stabilizationBatch",value:function(){this.startedStabilization===!1&&(this.body.emitter.emit("startStabilizing"),this.startedStabilization=!0);for(var t=0;this.stabilized===!1&&t0){var t=void 0,e=this.body.nodes,i=this.physicsBody.physicsNodeIndices,o=i.length,n=this._formBarnesHutTree(e,i);this.barnesHutTree=n;for(var s=0;o>s;s++)t=e[i[s]],t.options.mass>0&&(this._getForceContribution(n.root.children.NW,t),this._getForceContribution(n.root.children.NE,t),this._getForceContribution(n.root.children.SW,t),this._getForceContribution(n.root.children.SE,t))}}},{key:"_getForceContribution",value:function(t,e){if(t.childrenCount>0){var i=void 0,o=void 0,n=void 0;i=t.centerOfMass.x-e.x,o=t.centerOfMass.y-e.y,n=Math.sqrt(i*i+o*o),n*t.calcSize>this.thetaInversed?this._calculateForces(n,i,o,e,t):4===t.childrenCount?(this._getForceContribution(t.children.NW,e),this._getForceContribution(t.children.NE,e),this._getForceContribution(t.children.SW,e),this._getForceContribution(t.children.SE,e)):t.children.data.id!=e.id&&this._calculateForces(n,i,o,e,t)}}},{key:"_calculateForces",value:function(t,e,i,o,n){0===t&&(t=.1,e=t),this.overlapAvoidanceFactor<1&&(t=Math.max(.1+this.overlapAvoidanceFactor*o.shape.radius,t-o.shape.radius));var s=this.options.gravitationalConstant*n.mass*o.options.mass/Math.pow(t,3),r=e*s,a=i*s;this.physicsBody.forces[o.id].x+=r,this.physicsBody.forces[o.id].y+=a}},{key:"_formBarnesHutTree",value:function(t,e){for(var i=void 0,o=e.length,n=t[e[0]].x,s=t[e[0]].y,r=t[e[0]].x,a=t[e[0]].y,h=1;o>h;h++){var d=t[e[h]].x,l=t[e[h]].y;t[e[h]].options.mass>0&&(n>d&&(n=d),d>r&&(r=d),s>l&&(s=l),l>a&&(a=l))}var c=Math.abs(r-n)-Math.abs(a-s);c>0?(s-=.5*c,a+=.5*c):(n+=.5*c,r-=.5*c);var u=1e-5,p=Math.max(u,Math.abs(r-n)),f=.5*p,m=.5*(n+r),v=.5*(s+a),g={root:{centerOfMass:{x:0,y:0},mass:0,range:{minX:m-f,maxX:m+f,minY:v-f,maxY:v+f},size:p,calcSize:1/p,children:{data:null},maxWidth:0,level:0,childrenCount:4}};this._splitBranch(g.root);for(var y=0;o>y;y++)i=t[e[y]],i.options.mass>0&&this._placeInTree(g.root,i);return g}},{key:"_updateBranchMass",value:function(t,e){var i=t.mass+e.options.mass,o=1/i;t.centerOfMass.x=t.centerOfMass.x*t.mass+e.x*e.options.mass,t.centerOfMass.x*=o,t.centerOfMass.y=t.centerOfMass.y*t.mass+e.y*e.options.mass,t.centerOfMass.y*=o,t.mass=i;var n=Math.max(Math.max(e.height,e.radius),e.width);t.maxWidth=t.maxWidthe.x?t.children.NW.range.maxY>e.y?this._placeInRegion(t,e,"NW"):this._placeInRegion(t,e,"SW"):t.children.NW.range.maxY>e.y?this._placeInRegion(t,e,"NE"):this._placeInRegion(t,e,"SE")}},{key:"_placeInRegion",value:function(t,e,i){switch(t.children[i].childrenCount){case 0:t.children[i].children.data=e,t.children[i].childrenCount=1,this._updateBranchMass(t.children[i],e);break;case 1:t.children[i].children.data.x===e.x&&t.children[i].children.data.y===e.y?(e.x+=this.seededRandom(),e.y+=this.seededRandom()):(this._splitBranch(t.children[i]),this._placeInTree(t.children[i],e));break;case 4:this._placeInTree(t.children[i],e)}}},{key:"_splitBranch",value:function(t){var e=null;1===t.childrenCount&&(e=t.children.data,t.mass=0,t.centerOfMass.x=0,t.centerOfMass.y=0),t.childrenCount=4,t.children.data=null,this._insertRegion(t,"NW"),this._insertRegion(t,"NE"),this._insertRegion(t,"SW"),this._insertRegion(t,"SE"),null!=e&&this._placeInTree(t,e)}},{key:"_insertRegion",value:function(t,e){var i=void 0,o=void 0,n=void 0,s=void 0,r=.5*t.size;switch(e){case"NW":i=t.range.minX,o=t.range.minX+r,n=t.range.minY,s=t.range.minY+r;break;case"NE":i=t.range.minX+r,o=t.range.maxX,n=t.range.minY,s=t.range.minY+r;break;case"SW":i=t.range.minX,o=t.range.minX+r,n=t.range.minY+r,s=t.range.maxY;break;case"SE":i=t.range.minX+r,o=t.range.maxX,n=t.range.minY+r,s=t.range.maxY}t.children[e]={centerOfMass:{x:0,y:0},mass:0,range:{minX:i,maxX:o,minY:n,maxY:s},size:.5*t.size,calcSize:2*t.calcSize,children:{data:null},maxWidth:0,level:t.level+1,childrenCount:0}}},{key:"_debug",value:function(t,e){void 0!==this.barnesHutTree&&(t.lineWidth=1,this._drawBranch(this.barnesHutTree.root,t,e))}},{key:"_drawBranch",value:function(t,e,i){void 0===i&&(i="#FF0000"),4===t.childrenCount&&(this._drawBranch(t.children.NW,e),this._drawBranch(t.children.NE,e),this._drawBranch(t.children.SE,e),this._drawBranch(t.children.SW,e)),e.strokeStyle=i,e.beginPath(),e.moveTo(t.range.minX,t.range.minY),e.lineTo(t.range.maxX,t.range.minY),e.stroke(),e.beginPath(),e.moveTo(t.range.maxX,t.range.minY),e.lineTo(t.range.maxX,t.range.maxY),e.stroke(),e.beginPath(),e.moveTo(t.range.maxX,t.range.maxY), +e.lineTo(t.range.minX,t.range.maxY),e.stroke(),e.beginPath(),e.moveTo(t.range.minX,t.range.maxY),e.lineTo(t.range.minX,t.range.minY),e.stroke()}}]),t}();e["default"]=n},function(t,e){function i(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var o=function(){function t(t,e){for(var i=0;ii&&(s=.5*c>i?1:u*i+p,s/=i,o=t*s,n=e*s,l[r.id].x-=o,l[r.id].y-=n,l[a.id].x+=o,l[a.id].y+=n)}}}]),t}();e["default"]=n},function(t,e){function i(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var o=function(){function t(t,e){for(var i=0;ii?-Math.pow(f*i,2)+Math.pow(f*p,2):0,0===i?i=.01:s/=i,o=t*s,n=e*s,u[r.id].x-=o,u[r.id].y-=n,u[a.id].x+=o,u[a.id].y+=n}}}]),t}();e["default"]=n},function(t,e){function i(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var o=function(){function t(t,e){for(var i=0;i0){var s=n.edges.length+1,r=this.options.centralGravity*s*n.options.mass;o[n.id].x=e*r,o[n.id].y=i*r}}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},r=function(){function t(t,e){for(var i=0;i=t&&i.push(n.id)}for(var r=0;r0&&Object.keys(p).length>0&&m===!0&&o.push({nodes:u,edges:p})}}}for(var b=0;bo?r.x:o,n=r.ys?r.y:s;return{x:.5*(i+o),y:.5*(n+s)}}},{key:"openCluster",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!0:arguments[2];if(void 0===t)throw new Error("No clusterNodeId supplied to openCluster.");if(void 0===this.body.nodes[t])throw new Error("The clusterNodeId supplied to openCluster does not exist.");if(void 0===this.body.nodes[t].containedNodes)return void console.log("The node:"+t+" is not a cluster.");var o=this.body.nodes[t],n=o.containedNodes,s=o.containedEdges;if(void 0!==e&&void 0!==e.releaseFunction&&"function"==typeof e.releaseFunction){var r={},a={x:o.x,y:o.y};for(var d in n)if(n.hasOwnProperty(d)){var l=this.body.nodes[d];r[d]={x:l.x,y:l.y}}var u=e.releaseFunction(a,r);for(var p in n)if(n.hasOwnProperty(p)){var f=this.body.nodes[p];void 0!==u[p]&&(f.x=void 0===u[p].x?o.x:u[p].x,f.y=void 0===u[p].y?o.y:u[p].y)}}else for(var m in n)if(n.hasOwnProperty(m)){var v=this.body.nodes[m];v=n[m],v.options.fixed.x===!1&&(v.x=o.x),v.options.fixed.y===!1&&(v.y=o.y)}for(var g in n)if(n.hasOwnProperty(g)){var y=this.body.nodes[g];y.vx=o.vx,y.vy=o.vy,y.setOptions({hidden:!1,physics:!0}),delete this.clusteredNodes[g]}for(var b=[],w=0;wo;)e.push(this.body.nodes[t].id),t=this.clusteredNodes[t].clusterId,o++;return e.push(this.body.nodes[t].id),e.reverse(),e}},{key:"_getConnectedId",value:function(t,e){return t.toId!=e?t.toId:t.fromId!=e?t.fromId:t.fromId}},{key:"_getHubSize",value:function(){for(var t=0,e=0,i=0,o=0,n=0;no&&(o=s.edges.length),t+=s.edges.length,e+=Math.pow(s.edges.length,2),i+=1}t/=i,e/=i;var r=e-Math.pow(t,2),a=Math.sqrt(r),h=Math.floor(t+2*a);return h>o&&(h=o),h}}]),t}();e["default"]=u},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n=function(){function t(t,e){for(var i=0;i0)for(var a=0;ae.shape.boundingBox.left&&(s=e.shape.boundingBox.left),re.shape.boundingBox.top&&(o=e.shape.boundingBox.top),n0)for(var a=0;ae.x&&(s=e.x),re.y&&(o=e.y),n0,t.renderTimer=void 0}),this.body.emitter.on("destroy",function(){t.renderRequests=0,t.allowRedraw=!1,t.renderingActive=!1,t.requiresTimeout===!0?clearTimeout(t.renderTimer):cancelAnimationFrame(t.renderTimer),t.body.emitter.off()})}},{key:"setOptions",value:function(t){if(void 0!==t){var e=["hideEdgesOnDrag","hideNodesOnDrag"];s.selectiveDeepExtend(e,this.options,t)}}},{key:"_startRendering",value:function(){this.renderingActive===!0&&void 0===this.renderTimer&&(this.requiresTimeout===!0?this.renderTimer=window.setTimeout(this._renderStep.bind(this),this.simulationInterval):this.renderTimer=window.requestAnimationFrame(this._renderStep.bind(this)))}},{key:"_renderStep",value:function(){this.renderingActive===!0&&(this.renderTimer=void 0,this.requiresTimeout===!0&&this._startRendering(),this._redraw(),this.requiresTimeout===!1&&this._startRendering())}},{key:"redraw",value:function(){this.body.emitter.emit("setSize"),this._redraw()}},{key:"_requestRedraw",value:function(){var t=this;this.redrawRequested!==!0&&this.renderingActive===!1&&this.allowRedraw===!0&&(this.redrawRequested=!0,this.requiresTimeout===!0?window.setTimeout(function(){t._redraw(!1)},0):window.requestAnimationFrame(function(){t._redraw(!1)}))}},{key:"_redraw",value:function(){var t=arguments.length<=0||void 0===arguments[0]?!1:arguments[0];if(this.allowRedraw===!0){this.body.emitter.emit("initRedraw"),this.redrawRequested=!1;var e=this.canvas.frame.canvas.getContext("2d");0!==this.canvas.frame.canvas.width&&0!==this.canvas.frame.canvas.height||this.canvas.setSize(),this.pixelRatio=(window.devicePixelRatio||1)/(e.webkitBackingStorePixelRatio||e.mozBackingStorePixelRatio||e.msBackingStorePixelRatio||e.oBackingStorePixelRatio||e.backingStorePixelRatio||1),e.setTransform(this.pixelRatio,0,0,this.pixelRatio,0,0);var i=this.canvas.frame.canvas.clientWidth,o=this.canvas.frame.canvas.clientHeight;if(e.clearRect(0,0,i,o),0===this.canvas.frame.clientWidth)return;e.save(),e.translate(this.body.view.translation.x,this.body.view.translation.y),e.scale(this.body.view.scale,this.body.view.scale),e.beginPath(),this.body.emitter.emit("beforeDrawing",e),e.closePath(),t===!1&&(this.dragging===!1||this.dragging===!0&&this.options.hideEdgesOnDrag===!1)&&this._drawEdges(e),(this.dragging===!1||this.dragging===!0&&this.options.hideNodesOnDrag===!1)&&this._drawNodes(e,t),e.beginPath(),this.body.emitter.emit("afterDrawing",e),e.closePath(),e.restore(),t===!0&&e.clearRect(0,0,i,o)}}},{key:"_resizeNodes",value:function(){var t=this.canvas.frame.canvas.getContext("2d");void 0===this.pixelRatio&&(this.pixelRatio=(window.devicePixelRatio||1)/(t.webkitBackingStorePixelRatio||t.mozBackingStorePixelRatio||t.msBackingStorePixelRatio||t.oBackingStorePixelRatio||t.backingStorePixelRatio||1)),t.setTransform(this.pixelRatio,0,0,this.pixelRatio,0,0),t.save(),t.translate(this.body.view.translation.x,this.body.view.translation.y),t.scale(this.body.view.scale,this.body.view.scale);var e=this.body.nodes,i=void 0;for(var o in e)e.hasOwnProperty(o)&&(i=e[o],i.resize(t),i.updateBoundingBox(t,i.selected));t.restore()}},{key:"_drawNodes",value:function(t){for(var e=arguments.length<=1||void 0===arguments[1]?!1:arguments[1],i=this.body.nodes,o=this.body.nodeIndices,n=void 0,s=[],r=20,a=this.canvas.DOMtoCanvas({x:-r,y:-r}),h=this.canvas.DOMtoCanvas({x:this.canvas.frame.canvas.clientWidth+r,y:this.canvas.frame.canvas.clientHeight+r}),d={top:a.y,left:a.x,bottom:h.y,right:h.x},l=0;l0){var t=this.frame.canvas.width/this.pixelRatio/this.cameraState.previousWidth,e=this.frame.canvas.height/this.pixelRatio/this.cameraState.previousHeight,i=this.cameraState.scale;1!=t&&1!=e?i=.5*this.cameraState.scale*(t+e):1!=t?i=this.cameraState.scale*t:1!=e&&(i=this.cameraState.scale*e),this.body.view.scale=i;var o=this.DOMtoCanvas({x:.5*this.frame.canvas.clientWidth,y:.5*this.frame.canvas.clientHeight}),n={x:o.x-this.cameraState.position.x,y:o.y-this.cameraState.position.y};this.body.view.translation.x+=n.x*this.body.view.scale,this.body.view.translation.y+=n.y*this.body.view.scale}}},{key:"_prepareValue",value:function(t){if("number"==typeof t)return t+"px";if("string"==typeof t){if(-1!==t.indexOf("%")||-1!==t.indexOf("px"))return t;if(-1===t.indexOf("%"))return t+"px"}throw new Error("Could not use the value supplied for width or height:"+t)}},{key:"_create",value:function(){for(;this.body.container.hasChildNodes();)this.body.container.removeChild(this.body.container.firstChild);if(this.frame=document.createElement("div"),this.frame.className="vis-network",this.frame.style.position="relative",this.frame.style.overflow="hidden",this.frame.tabIndex=900,this.frame.canvas=document.createElement("canvas"),this.frame.canvas.style.position="relative",this.frame.appendChild(this.frame.canvas),this.frame.canvas.getContext){var t=this.frame.canvas.getContext("2d");this.pixelRatio=(window.devicePixelRatio||1)/(t.webkitBackingStorePixelRatio||t.mozBackingStorePixelRatio||t.msBackingStorePixelRatio||t.oBackingStorePixelRatio||t.backingStorePixelRatio||1), +this.frame.canvas.getContext("2d").setTransform(this.pixelRatio,0,0,this.pixelRatio,0,0)}else{var e=document.createElement("DIV");e.style.color="red",e.style.fontWeight="bold",e.style.padding="10px",e.innerHTML="Error: your browser does not support HTML canvas",this.frame.canvas.appendChild(e)}this.body.container.appendChild(this.frame),this.body.view.scale=1,this.body.view.translation={x:.5*this.frame.canvas.clientWidth,y:.5*this.frame.canvas.clientHeight},this._bindHammer()}},{key:"_bindHammer",value:function(){var t=this;void 0!==this.hammer&&this.hammer.destroy(),this.drag={},this.pinch={},this.hammer=new s(this.frame.canvas),this.hammer.get("pinch").set({enable:!0}),this.hammer.get("pan").set({threshold:5,direction:s.DIRECTION_ALL}),r.onTouch(this.hammer,function(e){t.body.eventListeners.onTouch(e)}),this.hammer.on("tap",function(e){t.body.eventListeners.onTap(e)}),this.hammer.on("doubletap",function(e){t.body.eventListeners.onDoubleTap(e)}),this.hammer.on("press",function(e){t.body.eventListeners.onHold(e)}),this.hammer.on("panstart",function(e){t.body.eventListeners.onDragStart(e)}),this.hammer.on("panmove",function(e){t.body.eventListeners.onDrag(e)}),this.hammer.on("panend",function(e){t.body.eventListeners.onDragEnd(e)}),this.hammer.on("pinch",function(e){t.body.eventListeners.onPinch(e)}),this.frame.canvas.addEventListener("mousewheel",function(e){t.body.eventListeners.onMouseWheel(e)}),this.frame.canvas.addEventListener("DOMMouseScroll",function(e){t.body.eventListeners.onMouseWheel(e)}),this.frame.canvas.addEventListener("mousemove",function(e){t.body.eventListeners.onMouseMove(e)}),this.frame.canvas.addEventListener("contextmenu",function(e){t.body.eventListeners.onContext(e)}),this.hammerFrame=new s(this.frame),r.onRelease(this.hammerFrame,function(e){t.body.eventListeners.onRelease(e)})}},{key:"setSize",value:function(){var t=arguments.length<=0||void 0===arguments[0]?this.options.width:arguments[0],e=arguments.length<=1||void 0===arguments[1]?this.options.height:arguments[1];t=this._prepareValue(t),e=this._prepareValue(e);var i=!1,o=this.frame.canvas.width,n=this.frame.canvas.height,s=this.frame.canvas.getContext("2d"),r=this.pixelRatio;return this.pixelRatio=(window.devicePixelRatio||1)/(s.webkitBackingStorePixelRatio||s.mozBackingStorePixelRatio||s.msBackingStorePixelRatio||s.oBackingStorePixelRatio||s.backingStorePixelRatio||1),t!=this.options.width||e!=this.options.height||this.frame.style.width!=t||this.frame.style.height!=e?(this._getCameraState(r),this.frame.style.width=t,this.frame.style.height=e,this.frame.canvas.style.width="100%",this.frame.canvas.style.height="100%",this.frame.canvas.width=Math.round(this.frame.canvas.clientWidth*this.pixelRatio),this.frame.canvas.height=Math.round(this.frame.canvas.clientHeight*this.pixelRatio),this.options.width=t,this.options.height=e,i=!0):(this.frame.canvas.width==Math.round(this.frame.canvas.clientWidth*this.pixelRatio)&&this.frame.canvas.height==Math.round(this.frame.canvas.clientHeight*this.pixelRatio)||this._getCameraState(r),this.frame.canvas.width!=Math.round(this.frame.canvas.clientWidth*this.pixelRatio)&&(this.frame.canvas.width=Math.round(this.frame.canvas.clientWidth*this.pixelRatio),i=!0),this.frame.canvas.height!=Math.round(this.frame.canvas.clientHeight*this.pixelRatio)&&(this.frame.canvas.height=Math.round(this.frame.canvas.clientHeight*this.pixelRatio),i=!0)),i===!0&&(this.body.emitter.emit("resize",{width:Math.round(this.frame.canvas.width/this.pixelRatio),height:Math.round(this.frame.canvas.height/this.pixelRatio),oldWidth:Math.round(o/this.pixelRatio),oldHeight:Math.round(n/this.pixelRatio)}),this._setCameraState()),this.initialized=!0,i}},{key:"_XconvertDOMtoCanvas",value:function(t){return(t-this.body.view.translation.x)/this.body.view.scale}},{key:"_XconvertCanvasToDOM",value:function(t){return t*this.body.view.scale+this.body.view.translation.x}},{key:"_YconvertDOMtoCanvas",value:function(t){return(t-this.body.view.translation.y)/this.body.view.scale}},{key:"_YconvertCanvasToDOM",value:function(t){return t*this.body.view.scale+this.body.view.translation.y}},{key:"canvasToDOM",value:function(t){return{x:this._XconvertCanvasToDOM(t.x),y:this._YconvertCanvasToDOM(t.y)}}},{key:"DOMtoCanvas",value:function(t){return{x:this._XconvertDOMtoCanvas(t.x),y:this._YconvertDOMtoCanvas(t.y)}}}]),t}();e["default"]=h},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s=function(){function t(t,e){for(var i=0;i.5*this.body.nodeIndices.length)return void this.fit(t,!1);i=a["default"].getRange(this.body.nodes,t.nodes);var h=this.body.nodeIndices.length;o=12.662/(h+7.4147)+.0964822;var d=Math.min(this.canvas.frame.canvas.clientWidth/600,this.canvas.frame.canvas.clientHeight/600);o*=d}else{this.body.emitter.emit("_resizeNodes"),i=a["default"].getRange(this.body.nodes,t.nodes);var l=1.1*Math.abs(i.maxX-i.minX),c=1.1*Math.abs(i.maxY-i.minY),u=this.canvas.frame.canvas.clientWidth/l,p=this.canvas.frame.canvas.clientHeight/c;o=p>=u?u:p}o>1?o=1:0===o&&(o=1);var f=a["default"].findCenter(i),m={position:f,scale:o,animation:t.animation};this.moveTo(m)}},{key:"focus",value:function(t){var e=arguments.length<=1||void 0===arguments[1]?{}:arguments[1];if(void 0!==this.body.nodes[t]){var i={x:this.body.nodes[t].x,y:this.body.nodes[t].y};e.position=i,e.lockedOnNode=t,this.moveTo(e)}else console.log("Node: "+t+" cannot be found.")}},{key:"moveTo",value:function(t){return void 0===t?void(t={}):(void 0===t.offset&&(t.offset={x:0,y:0}),void 0===t.offset.x&&(t.offset.x=0),void 0===t.offset.y&&(t.offset.y=0),void 0===t.scale&&(t.scale=this.body.view.scale),void 0===t.position&&(t.position=this.getViewPosition()),void 0===t.animation&&(t.animation={duration:0}),t.animation===!1&&(t.animation={duration:0}),t.animation===!0&&(t.animation={}),void 0===t.animation.duration&&(t.animation.duration=1e3),void 0===t.animation.easingFunction&&(t.animation.easingFunction="easeInOutQuad"),void this.animateView(t))}},{key:"animateView",value:function(t){if(void 0!==t){this.animationEasingFunction=t.animation.easingFunction,this.releaseNode(),t.locked===!0&&(this.lockedOnNodeId=t.lockedOnNode,this.lockedOnNodeOffset=t.offset),0!=this.easingTime&&this._transitionRedraw(!0),this.sourceScale=this.body.view.scale,this.sourceTranslation=this.body.view.translation,this.targetScale=t.scale,this.body.view.scale=this.targetScale;var e=this.canvas.DOMtoCanvas({x:.5*this.canvas.frame.canvas.clientWidth,y:.5*this.canvas.frame.canvas.clientHeight}),i={x:e.x-t.position.x,y:e.y-t.position.y};this.targetTranslation={x:this.sourceTranslation.x+i.x*this.targetScale+t.offset.x,y:this.sourceTranslation.y+i.y*this.targetScale+t.offset.y},0===t.animation.duration?void 0!=this.lockedOnNodeId?(this.viewFunction=this._lockedRedraw.bind(this),this.body.emitter.on("initRedraw",this.viewFunction)):(this.body.view.scale=this.targetScale,this.body.view.translation=this.targetTranslation,this.body.emitter.emit("_requestRedraw")):(this.animationSpeed=1/(60*t.animation.duration*.001)||1/60,this.animationEasingFunction=t.animation.easingFunction,this.viewFunction=this._transitionRedraw.bind(this),this.body.emitter.on("initRedraw",this.viewFunction),this.body.emitter.emit("_startRendering"))}}},{key:"_lockedRedraw",value:function(){var t={x:this.body.nodes[this.lockedOnNodeId].x,y:this.body.nodes[this.lockedOnNodeId].y},e=this.canvas.DOMtoCanvas({x:.5*this.canvas.frame.canvas.clientWidth,y:.5*this.canvas.frame.canvas.clientHeight}),i={x:e.x-t.x,y:e.y-t.y},o=this.body.view.translation,n={x:o.x+i.x*this.body.view.scale+this.lockedOnNodeOffset.x,y:o.y+i.y*this.body.view.scale+this.lockedOnNodeOffset.y};this.body.view.translation=n}},{key:"releaseNode",value:function(){void 0!==this.lockedOnNodeId&&void 0!==this.viewFunction&&(this.body.emitter.off("initRedraw",this.viewFunction),this.lockedOnNodeId=void 0,this.lockedOnNodeOffset=void 0)}},{key:"_transitionRedraw",value:function(){var t=arguments.length<=0||void 0===arguments[0]?!1:arguments[0];this.easingTime+=this.animationSpeed,this.easingTime=t===!0?1:this.easingTime;var e=h.easingFunctions[this.animationEasingFunction](this.easingTime);this.body.view.scale=this.sourceScale+(this.targetScale-this.sourceScale)*e,this.body.view.translation={x:this.sourceTranslation.x+(this.targetTranslation.x-this.sourceTranslation.x)*e,y:this.sourceTranslation.y+(this.targetTranslation.y-this.sourceTranslation.y)*e},this.easingTime>=1&&(this.body.emitter.off("initRedraw",this.viewFunction),this.easingTime=0,void 0!=this.lockedOnNodeId&&(this.viewFunction=this._lockedRedraw.bind(this),this.body.emitter.on("initRedraw",this.viewFunction)),this.body.emitter.emit("animationFinished"))}},{key:"getScale",value:function(){return this.body.view.scale}},{key:"getViewPosition",value:function(){return this.canvas.DOMtoCanvas({x:.5*this.canvas.frame.canvas.clientWidth,y:.5*this.canvas.frame.canvas.clientHeight})}}]),t}();e["default"]=d},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s=function(){function t(t,e){for(var i=0;i50&&(this.drag.pointer=this.getPointer(t.center),this.drag.pinched=!1,this.pinch.scale=this.body.view.scale,this.touchTime=(new Date).valueOf())}},{key:"onTap",value:function(t){var e=this.getPointer(t.center),i=this.selectionHandler.options.multiselect&&(t.changedPointers[0].ctrlKey||t.changedPointers[0].metaKey);this.checkSelectionChanges(e,t,i),this.selectionHandler._generateClickEvent("click",t,e)}},{key:"onDoubleTap",value:function(t){var e=this.getPointer(t.center);this.selectionHandler._generateClickEvent("doubleClick",t,e)}},{key:"onHold",value:function(t){var e=this.getPointer(t.center),i=this.selectionHandler.options.multiselect;this.checkSelectionChanges(e,t,i),this.selectionHandler._generateClickEvent("click",t,e),this.selectionHandler._generateClickEvent("hold",t,e)}},{key:"onRelease",value:function(t){if((new Date).valueOf()-this.touchTime>10){var e=this.getPointer(t.center);this.selectionHandler._generateClickEvent("release",t,e),this.touchTime=(new Date).valueOf()}}},{key:"onContext",value:function(t){var e=this.getPointer({x:t.clientX,y:t.clientY});this.selectionHandler._generateClickEvent("oncontext",t,e)}},{key:"checkSelectionChanges",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!1:arguments[2],o=this.selectionHandler._getSelectedEdgeCount(),n=this.selectionHandler._getSelectedNodeCount(),s=this.selectionHandler.getSelection(),r=void 0;r=i===!0?this.selectionHandler.selectAdditionalOnPoint(t):this.selectionHandler.selectOnPoint(t);var a=this.selectionHandler._getSelectedEdgeCount(),h=this.selectionHandler._getSelectedNodeCount(),d=this.selectionHandler.getSelection(),l=this._determineIfDifferent(s,d),c=l.nodesChanged,u=l.edgesChanged,p=!1;h-n>0?(this.selectionHandler._generateClickEvent("selectNode",e,t),r=!0,p=!0):c===!0&&h>0?(this.selectionHandler._generateClickEvent("deselectNode",e,t,s),this.selectionHandler._generateClickEvent("selectNode",e,t),p=!0,r=!0):0>h-n&&(this.selectionHandler._generateClickEvent("deselectNode",e,t,s),r=!0),a-o>0&&p===!1?(this.selectionHandler._generateClickEvent("selectEdge",e,t),r=!0):a>0&&u===!0?(this.selectionHandler._generateClickEvent("deselectEdge",e,t,s),this.selectionHandler._generateClickEvent("selectEdge",e,t),r=!0):0>a-o&&(this.selectionHandler._generateClickEvent("deselectEdge",e,t,s),r=!0),r===!0&&this.selectionHandler._generateClickEvent("select",e,t)}},{key:"_determineIfDifferent",value:function(t,e){for(var i=!1,o=!1,n=0;nt&&(t=1e-5),t>10&&(t=10);var o=void 0;void 0!==this.drag&&this.drag.dragging===!0&&(o=this.canvas.DOMtoCanvas(this.drag.pointer));var n=this.body.view.translation,s=t/i,r=(1-s)*e.x+n.x*s,a=(1-s)*e.y+n.y*s;if(this.body.view.scale=t,this.body.view.translation={x:r,y:a},void 0!=o){var h=this.canvas.canvasToDOM(o);this.drag.pointer.x=h.x,this.drag.pointer.y=h.y}this.body.emitter.emit("_requestRedraw"),t>i?this.body.emitter.emit("zoom",{direction:"+",scale:this.body.view.scale}):this.body.emitter.emit("zoom",{direction:"-",scale:this.body.view.scale})}}},{key:"onMouseWheel",value:function(t){if(this.options.zoomView===!0){var e=0;if(t.wheelDelta?e=t.wheelDelta/120:t.detail&&(e=-t.detail/3),0!==e){var i=this.body.view.scale,o=e/10;0>e&&(o/=1-o),i*=1+o;var n=this.getPointer({x:t.clientX,y:t.clientY});this.zoom(i,n)}t.preventDefault()}}},{key:"onMouseMove",value:function(t){var e=this,i=this.getPointer({x:t.clientX,y:t.clientY}),o=!1;if(void 0!==this.popup&&(this.popup.hidden===!1&&this._checkHidePopup(i),this.popup.hidden===!1&&(o=!0,this.popup.setPosition(i.x+3,i.y-5),this.popup.show())),this.options.keyboard.bindToWindow===!1&&this.options.keyboard.enabled===!0&&this.canvas.frame.focus(),o===!1&&(void 0!==this.popupTimer&&(clearInterval(this.popupTimer),this.popupTimer=void 0),this.drag.dragging||(this.popupTimer=setTimeout(function(){return e._checkShowPopup(i)},this.options.tooltipDelay))),this.options.hover===!0){var n=this.selectionHandler.getNodeAt(i);void 0===n&&(n=this.selectionHandler.getEdgeAt(i)),this.selectionHandler.hoverObject(n)}}},{key:"_checkShowPopup",value:function(t){var e=this.canvas._XconvertDOMtoCanvas(t.x),i=this.canvas._YconvertDOMtoCanvas(t.y),o={left:e,top:i,right:e,bottom:i},n=void 0===this.popupObj?void 0:this.popupObj.id,s=!1,r="node";if(void 0===this.popupObj){for(var a=this.body.nodeIndices,h=this.body.nodes,l=void 0,c=[],u=0;u0&&(this.popupObj=h[c[c.length-1]],s=!0)}if(void 0===this.popupObj&&s===!1){for(var p=this.body.edgeIndices,f=this.body.edges,m=void 0,v=[],g=0;g0&&(this.popupObj=f[v[v.length-1]],r="edge")}void 0!==this.popupObj?this.popupObj.id!==n&&(void 0===this.popup&&(this.popup=new d["default"](this.canvas.frame)),this.popup.popupTargetType=r,this.popup.popupTargetId=this.popupObj.id,this.popup.setPosition(t.x+3,t.y-5),this.popup.setText(this.popupObj.getTitle()),this.popup.show(),this.body.emitter.emit("showPopup",this.popupObj.id)):void 0!==this.popup&&(this.popup.hide(),this.body.emitter.emit("hidePopup"))}},{key:"_checkHidePopup",value:function(t){var e=this.selectionHandler._pointerToPositionObject(t),i=!1;if("node"===this.popup.popupTargetType){if(void 0!==this.body.nodes[this.popup.popupTargetId]&&(i=this.body.nodes[this.popup.popupTargetId].isOverlappingWith(e),i===!0)){var o=this.selectionHandler.getNodeAt(t);i=o.id===this.popup.popupTargetId}}else void 0===this.selectionHandler.getNodeAt(t)&&void 0!==this.body.edges[this.popup.popupTargetId]&&(i=this.body.edges[this.popup.popupTargetId].isOverlappingWith(e));i===!1&&(this.popupObj=void 0,this.popup.hide(),this.body.emitter.emit("hidePopup"))}}]),t}();e["default"]=c},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n=function(){function t(t,e){for(var i=0;i700&&(this.body.emitter.emit("fit",{duration:700}),this.touchTime=(new Date).valueOf())}},{key:"_stopMovement",value:function(){for(var t in this.boundFunctions)this.boundFunctions.hasOwnProperty(t)&&(this.body.emitter.off("initRedraw",this.boundFunctions[t]),this.body.emitter.emit("_stopRendering"));this.boundFunctions={}}},{key:"_moveUp",value:function(){this.body.view.translation.y+=this.options.keyboard.speed.y}},{key:"_moveDown",value:function(){this.body.view.translation.y-=this.options.keyboard.speed.y}},{key:"_moveLeft",value:function(){this.body.view.translation.x+=this.options.keyboard.speed.x}},{key:"_moveRight",value:function(){this.body.view.translation.x-=this.options.keyboard.speed.x}},{key:"_zoomIn",value:function(){this.body.view.scale*=1+this.options.keyboard.speed.zoom,this.body.emitter.emit("zoom",{direction:"+",scale:this.body.view.scale})}},{key:"_zoomOut",value:function(){this.body.view.scale/=1+this.options.keyboard.speed.zoom,this.body.emitter.emit("zoom",{direction:"-",scale:this.body.view.scale})}},{key:"configureKeyboardBindings",value:function(){var t=this;void 0!==this.keycharm&&this.keycharm.destroy(),this.options.keyboard.enabled===!0&&(this.options.keyboard.bindToWindow===!0?this.keycharm=a({container:window,preventDefault:!0}):this.keycharm=a({container:this.canvas.frame,preventDefault:!0}),this.keycharm.reset(),this.activated===!0&&(this.keycharm.bind("up",function(){t.bindToRedraw("_moveUp")},"keydown"),this.keycharm.bind("down",function(){t.bindToRedraw("_moveDown")},"keydown"),this.keycharm.bind("left",function(){t.bindToRedraw("_moveLeft")},"keydown"),this.keycharm.bind("right",function(){t.bindToRedraw("_moveRight")},"keydown"),this.keycharm.bind("=",function(){t.bindToRedraw("_zoomIn")},"keydown"),this.keycharm.bind("num+",function(){t.bindToRedraw("_zoomIn")},"keydown"),this.keycharm.bind("num-",function(){t.bindToRedraw("_zoomOut")},"keydown"),this.keycharm.bind("-",function(){t.bindToRedraw("_zoomOut")},"keydown"),this.keycharm.bind("[",function(){t.bindToRedraw("_zoomOut")},"keydown"),this.keycharm.bind("]",function(){t.bindToRedraw("_zoomIn")},"keydown"),this.keycharm.bind("pageup",function(){t.bindToRedraw("_zoomIn")},"keydown"),this.keycharm.bind("pagedown",function(){t.bindToRedraw("_zoomOut")},"keydown"),this.keycharm.bind("up",function(){t.unbindFromRedraw("_moveUp")},"keyup"),this.keycharm.bind("down",function(){t.unbindFromRedraw("_moveDown")},"keyup"),this.keycharm.bind("left",function(){t.unbindFromRedraw("_moveLeft")},"keyup"),this.keycharm.bind("right",function(){t.unbindFromRedraw("_moveRight")},"keyup"),this.keycharm.bind("=",function(){t.unbindFromRedraw("_zoomIn")},"keyup"),this.keycharm.bind("num+",function(){t.unbindFromRedraw("_zoomIn")},"keyup"),this.keycharm.bind("num-",function(){t.unbindFromRedraw("_zoomOut")},"keyup"),this.keycharm.bind("-",function(){t.unbindFromRedraw("_zoomOut")},"keyup"),this.keycharm.bind("[",function(){t.unbindFromRedraw("_zoomOut")},"keyup"),this.keycharm.bind("]",function(){t.unbindFromRedraw("_zoomIn")},"keyup"),this.keycharm.bind("pageup",function(){t.unbindFromRedraw("_zoomIn")},"keyup"),this.keycharm.bind("pagedown",function(){t.unbindFromRedraw("_zoomOut")},"keyup")))}}]),t}();e["default"]=h},function(t,e){function i(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var o=function(){function t(t,e){for(var i=0;io&&(s=o-e-this.padding),sn&&(r=n-i-this.padding),r0?e===!0?this.body.nodes[o[o.length-1]]:o[o.length-1]:void 0}},{key:"_getEdgesOverlappingWith",value:function(t,e){for(var i=this.body.edges,o=0;o0?e===!0?this.body.edges[o[o.length-1]]:o[o.length-1]:void 0}},{key:"_addToSelection",value:function(t){t instanceof a["default"]?this.selectionObj.nodes[t.id]=t:this.selectionObj.edges[t.id]=t}},{key:"_addToHover",value:function(t){t instanceof a["default"]?this.hoverObj.nodes[t.id]=t:this.hoverObj.edges[t.id]=t}},{key:"_removeFromSelection",value:function(t){t instanceof a["default"]?(delete this.selectionObj.nodes[t.id],this._unselectConnectedEdges(t)):delete this.selectionObj.edges[t.id]}},{key:"unselectAll",value:function(){for(var t in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(t)&&this.selectionObj.nodes[t].unselect();for(var e in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(e)&&this.selectionObj.edges[e].unselect();this.selectionObj={nodes:{},edges:{}}}},{key:"_getSelectedNodeCount",value:function(){var t=0;for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&(t+=1);return t}},{key:"_getSelectedNode",value:function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t))return this.selectionObj.nodes[t]}},{key:"_getSelectedEdge",value:function(){for(var t in this.selectionObj.edges)if(this.selectionObj.edges.hasOwnProperty(t))return this.selectionObj.edges[t]}},{key:"_getSelectedEdgeCount",value:function(){var t=0;for(var e in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(e)&&(t+=1);return t}},{key:"_getSelectedObjectCount",value:function(){var t=0;for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&(t+=1);for(var i in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(i)&&(t+=1);return t}},{key:"_selectionIsEmpty",value:function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t))return!1;for(var e in this.selectionObj.edges)if(this.selectionObj.edges.hasOwnProperty(e))return!1;return!0}},{key:"_clusterInSelection",value:function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t)&&this.selectionObj.nodes[t].clusterSize>1)return!0;return!1}},{key:"_selectConnectedEdges",value:function(t){for(var e=0;e0&&(this.options.hierarchical.levelSeparation*=-1):this.options.hierarchical.levelSeparation<0&&(this.options.hierarchical.levelSeparation*=-1),this.body.emitter.emit("_resetHierarchicalLayout"),this.adaptAllOptionsForHierarchicalLayout(e);if(i===!0)return this.body.emitter.emit("refresh"),l.deepExtend(e,this.optionsBackup)}return e}},{key:"adaptAllOptionsForHierarchicalLayout",value:function(t){if(this.options.hierarchical.enabled===!0){void 0===t.physics||t.physics===!0?(t.physics={enabled:void 0===this.optionsBackup.physics.enabled?!0:this.optionsBackup.physics.enabled,solver:"hierarchicalRepulsion"},this.optionsBackup.physics.enabled=void 0===this.optionsBackup.physics.enabled?!0:this.optionsBackup.physics.enabled,this.optionsBackup.physics.solver=this.optionsBackup.physics.solver||"barnesHut"):"object"===r(t.physics)?(this.optionsBackup.physics.enabled=void 0===t.physics.enabled?!0:t.physics.enabled,this.optionsBackup.physics.solver=t.physics.solver||"barnesHut",t.physics.solver="hierarchicalRepulsion"):t.physics!==!1&&(this.optionsBackup.physics.solver="barnesHut",t.physics={solver:"hierarchicalRepulsion"});var e="horizontal";"RL"!==this.options.hierarchical.direction&&"LR"!==this.options.hierarchical.direction||(e="vertical"),void 0===t.edges?(this.optionsBackup.edges={smooth:{enabled:!0,type:"dynamic"}},t.edges={smooth:!1}):void 0===t.edges.smooth?(this.optionsBackup.edges={smooth:{enabled:!0,type:"dynamic"}},t.edges.smooth=!1):"boolean"==typeof t.edges.smooth?(this.optionsBackup.edges={smooth:t.edges.smooth},t.edges.smooth={enabled:t.edges.smooth,type:e}):(void 0!==t.edges.smooth.type&&"dynamic"!==t.edges.smooth.type&&(e=t.edges.smooth.type),this.optionsBackup.edges={smooth:void 0===t.edges.smooth.enabled?!0:t.edges.smooth.enabled,type:void 0===t.edges.smooth.type?"dynamic":t.edges.smooth.type,roundness:void 0===t.edges.smooth.roundness?.5:t.edges.smooth.roundness,forceDirection:void 0===t.edges.smooth.forceDirection?!1:t.edges.smooth.forceDirection},t.edges.smooth={enabled:void 0===t.edges.smooth.enabled?!0:t.edges.smooth.enabled,type:e,roundness:void 0===t.edges.smooth.roundness?.5:t.edges.smooth.roundness,forceDirection:void 0===t.edges.smooth.forceDirection?!1:t.edges.smooth.forceDirection}),this.body.emitter.emit("_forceDisableDynamicCurves",e)}return t}},{key:"seededRandom",value:function(){var t=1e4*Math.sin(this.randomSeed++);return t-Math.floor(t)}},{key:"positionInitially",value:function(t){if(this.options.hierarchical.enabled!==!0){this.randomSeed=this.initialRandomSeed;for(var e=0;es){for(var r=this.body.nodeIndices.length;this.body.nodeIndices.length>s;){n+=1;var a=this.body.nodeIndices.length;n%3===0?this.body.modules.clustering.clusterBridges():this.body.modules.clustering.clusterOutliers();var h=this.body.nodeIndices.length;if(a==h&&n%3!==0||n>o)return this._declusterAll(),this.body.emitter.emit("_layoutFailed"),void console.info("This network could not be positioned by this version of the improved layout algorithm. Please disable improvedLayout for better performance.")}this.body.modules.kamadaKawai.setOptions({springLength:Math.max(150,2*r)})}this.body.modules.kamadaKawai.solve(this.body.nodeIndices,this.body.edgeIndices,!0),this._shiftToCenter();for(var d=70,l=0;l0){var t=void 0,e=void 0,i=!1,o=!0,n=!1;this.hierarchicalLevels={},this.lastNodeOnLevel={},this.hierarchicalChildrenReference={},this.hierarchicalParentReference={},this.hierarchicalTrees={},this.treeIndex=-1,this.distributionOrdering={},this.distributionIndex={},this.distributionOrderingPresence={};for(e in this.body.nodes)this.body.nodes.hasOwnProperty(e)&&(t=this.body.nodes[e],void 0===t.options.x&&void 0===t.options.y&&(o=!1),void 0!==t.options.level?(i=!0,this.hierarchicalLevels[e]=t.options.level):n=!0);if(n===!0&&i===!0)throw new Error("To use the hierarchical layout, nodes require either no predefined levels or levels have to be defined for all nodes.");n===!0&&("hubsize"===this.options.hierarchical.sortMethod?this._determineLevelsByHubsize():"directed"===this.options.hierarchical.sortMethod?this._determineLevelsDirected():"custom"===this.options.hierarchical.sortMethod&&this._determineLevelsCustomCallback());for(var s in this.body.nodes)this.body.nodes.hasOwnProperty(s)&&void 0===this.hierarchicalLevels[s]&&(this.hierarchicalLevels[s]=0);var r=this._getDistribution();this._generateMap(),this._placeNodesByHierarchy(r),this._condenseHierarchy(),this._shiftToCenter()}}},{key:"_condenseHierarchy",value:function(){var t=this,e=!1,i={},o=function(){for(var e=a(),i=0;i0)for(var n=0;n=l&&(r=Math.min(c,r),a=Math.max(c,a))}return[r,a,o,n]},l=function _(e){var i=t.hierarchicalLevels[e];if(t.hierarchicalChildrenReference[e]){var o=t.hierarchicalChildrenReference[e];if(o.length>0)for(var n=0;n1)for(var a=0;at.options.hierarchical.nodeSpacing){var u={};u[i.id]=!0;var p={};p[o.id]=!0,h(i,u),h(o,p);var f=c(i,o),m=d(u,f),v=s(m,4),g=(v[0],v[1]),y=(v[2],v[3],d(p,f)),b=s(y,4),w=b[0],_=(b[1],b[2]),x=(b[3],Math.abs(g-w));if(x>t.options.hierarchical.nodeSpacing){var k=g-w+t.options.hierarchical.nodeSpacing;k<-_+t.options.hierarchical.nodeSpacing&&(k=-_+t.options.hierarchical.nodeSpacing),0>k&&(t._shiftBlock(o.id,k),e=!0,n===!0&&t._centerParent(o))}}},m=function(o,n){for(var r=n.id,a=n.edges,l=t.hierarchicalLevels[n.id],c=t.options.hierarchical.levelSeparation*t.options.hierarchical.levelSeparation,u={},p=[],f=0;fr;r++){var a=g(o,i),h=y(o,i),d=40,l=Math.max(-d,Math.min(d,Math.round(a/h)));if(o-=l,void 0!==s[o])break;s[o]=r}return o},w=function(o){var r=t._getPositionForHierarchy(n);if(void 0===i[n.id]){var a={};a[n.id]=!0,h(n,a),i[n.id]=a}var l=d(i[n.id]),c=s(l,4),u=(c[0],c[1],c[2]),p=c[3],f=o-r,m=0;f>0?m=Math.min(f,p-t.options.hierarchical.nodeSpacing):0>f&&(m=-Math.min(-f,u-t.options.hierarchical.nodeSpacing)),0!=m&&(t._shiftBlock(n.id,m),e=!0)},_=function(i){var o=t._getPositionForHierarchy(n),r=t._getSpaceAroundNode(n),a=s(r,2),h=a[0],d=a[1],l=i-o,c=o;l>0?c=Math.min(o+(d-t.options.hierarchical.nodeSpacing),i):0>l&&(c=Math.max(o-(h-t.options.hierarchical.nodeSpacing),i)),c!==o&&(t._setPositionForHierarchy(n,c,void 0,!0),e=!0)},x=b(o,p);w(x),x=b(o,a),_(x)},v=function(i){var o=Object.keys(t.distributionOrdering);o=o.reverse();for(var n=0;i>n;n++){e=!1;for(var s=0;sn&&(e=!1,p(f,o,!0),e===!0);n++);},y=function(){for(var e in t.body.nodes)t.body.nodes.hasOwnProperty(e)&&t._centerParent(t.body.nodes[e])},b=function(){var e=Object.keys(t.distributionOrdering);e=e.reverse();for(var i=0;i0)for(var d=0;dg&&Math.abs(g)0&&Math.abs(g)0&&(r=this._getPositionForHierarchy(i[n-1])+this.options.hierarchical.nodeSpacing),this._setPositionForHierarchy(s,r,e),this._validataPositionAndContinue(s,e,r),o++}}}}},{key:"_placeBranchNodes",value:function(t,e){if(void 0!==this.hierarchicalChildrenReference[t]){for(var i=[],o=0;oe&&void 0===this.positionedNodes[s.id]))return;var a=void 0;a=0===n?this._getPositionForHierarchy(this.body.nodes[t]):this._getPositionForHierarchy(i[n-1])+this.options.hierarchical.nodeSpacing,this._setPositionForHierarchy(s,a,r),this._validataPositionAndContinue(s,r,a)}for(var h=1e9,d=-1e9,l=0;l0&&(e=this._getHubSize(),0!==e);)for(var o in this.body.nodes)if(this.body.nodes.hasOwnProperty(o)){var n=this.body.nodes[o];n.edges.length===e&&this._crawlNetwork(i,o)}}},{key:"_determineLevelsCustomCallback",value:function(){var t=this,e=1e5,i=function(t,e,i){},o=function(o,n,s){var r=t.hierarchicalLevels[o.id];void 0===r&&(t.hierarchicalLevels[o.id]=e);var a=i(d["default"].cloneOptions(o,"node"),d["default"].cloneOptions(n,"node"),d["default"].cloneOptions(s,"edge"));t.hierarchicalLevels[n.id]=t.hierarchicalLevels[o.id]+a};this._crawlNetwork(o),this._setMinLevelToZero()}},{key:"_determineLevelsDirected",value:function(){var t=this,e=1e4,i=function(i,o,n){var s=t.hierarchicalLevels[i.id];void 0===s&&(t.hierarchicalLevels[i.id]=e),n.toId==o.id?t.hierarchicalLevels[o.id]=t.hierarchicalLevels[i.id]+1:t.hierarchicalLevels[o.id]=t.hierarchicalLevels[i.id]-1};this._crawlNetwork(i),this._setMinLevelToZero()}},{key:"_setMinLevelToZero",value:function(){var t=1e9;for(var e in this.body.nodes)this.body.nodes.hasOwnProperty(e)&&void 0!==this.hierarchicalLevels[e]&&(t=Math.min(this.hierarchicalLevels[e],t));for(var i in this.body.nodes)this.body.nodes.hasOwnProperty(i)&&void 0!==this.hierarchicalLevels[i]&&(this.hierarchicalLevels[i]-=t)}},{key:"_generateMap",value:function(){var t=this,e=function(e,i){if(t.hierarchicalLevels[i.id]>t.hierarchicalLevels[e.id]){var o=e.id,n=i.id;void 0===t.hierarchicalChildrenReference[o]&&(t.hierarchicalChildrenReference[o]=[]),t.hierarchicalChildrenReference[o].push(n),void 0===t.hierarchicalParentReference[n]&&(t.hierarchicalParentReference[n]=[]),t.hierarchicalParentReference[n].push(o)}};this._crawlNetwork(e)}},{key:"_crawlNetwork",value:function(){var t=this,e=arguments.length<=0||void 0===arguments[0]?function(){}:arguments[0],i=arguments[1],o={},n=0,s=function d(i,n){if(void 0===o[i.id]){void 0===t.hierarchicalTrees[i.id]&&(t.hierarchicalTrees[i.id]=n,t.treeIndex=Math.max(n,t.treeIndex)),o[i.id]=!0;for(var s=void 0,r=0;r1&&("UD"===this.options.hierarchical.direction||"DU"===this.options.hierarchical.direction?t.sort(function(t,e){return t.x-e.x}):t.sort(function(t,e){return t.y-e.y}))}}]),t}();e["default"]=c},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n=function(){function t(t,e){for(var i=0;i0&&this.options.deleteNode!==!1?(n===!0&&this._createSeperator(4),this._createDeleteButton(o)):0===t&&this.options.deleteEdge!==!1&&(n===!0&&this._createSeperator(4),this._createDeleteButton(o))),this._bindHammerToDiv(this.closeDiv,this.toggleEditMode.bind(this)),this._temporaryBindEvent("select",this.showManipulatorToolbar.bind(this))}this.body.emitter.emit("_redraw")}},{key:"addNodeMode",value:function(){if(this.editMode!==!0&&this.enableEditMode(),this._clean(),this.inMode="addNode",this.guiEnabled===!0){var t=this.options.locales[this.options.locale];this.manipulationDOM={},this._createBackButton(t),this._createSeperator(),this._createDescription(t.addDescription||this.options.locales.en.addDescription),this._bindHammerToDiv(this.closeDiv,this.toggleEditMode.bind(this))}this._temporaryBindEvent("click",this._performAddNode.bind(this))}},{key:"editNode",value:function(){var t=this;this.editMode!==!0&&this.enableEditMode(),this._clean();var e=this.selectionHandler._getSelectedNode();if(void 0!==e){if(this.inMode="editNode","function"!=typeof this.options.editNode)throw new Error("No function has been configured to handle the editing of nodes.");if(e.isCluster!==!0){var i=s.deepExtend({},e.options,!1); +if(i.x=e.x,i.y=e.y,2!==this.options.editNode.length)throw new Error("The function for edit does not support two arguments (data, callback)");this.options.editNode(i,function(e){null!==e&&void 0!==e&&"editNode"===t.inMode&&t.body.data.nodes.getDataSet().update(e),t.showManipulatorToolbar()})}else alert(this.options.locales[this.options.locale].editClusterError||this.options.locales.en.editClusterError)}else this.showManipulatorToolbar()}},{key:"addEdgeMode",value:function(){if(this.editMode!==!0&&this.enableEditMode(),this._clean(),this.inMode="addEdge",this.guiEnabled===!0){var t=this.options.locales[this.options.locale];this.manipulationDOM={},this._createBackButton(t),this._createSeperator(),this._createDescription(t.edgeDescription||this.options.locales.en.edgeDescription),this._bindHammerToDiv(this.closeDiv,this.toggleEditMode.bind(this))}this._temporaryBindUI("onTouch",this._handleConnect.bind(this)),this._temporaryBindUI("onDragEnd",this._finishConnect.bind(this)),this._temporaryBindUI("onDrag",this._dragControlNode.bind(this)),this._temporaryBindUI("onRelease",this._finishConnect.bind(this)),this._temporaryBindUI("onDragStart",function(){}),this._temporaryBindUI("onHold",function(){})}},{key:"editEdgeMode",value:function(){var t=this;if(this.editMode!==!0&&this.enableEditMode(),this._clean(),this.inMode="editEdge",this.guiEnabled===!0){var e=this.options.locales[this.options.locale];this.manipulationDOM={},this._createBackButton(e),this._createSeperator(),this._createDescription(e.editEdgeDescription||this.options.locales.en.editEdgeDescription),this._bindHammerToDiv(this.closeDiv,this.toggleEditMode.bind(this))}this.edgeBeingEditedId=this.selectionHandler.getSelectedEdges()[0],void 0!==this.edgeBeingEditedId?!function(){var e=t.body.edges[t.edgeBeingEditedId],i=t._getNewTargetNode(e.from.x,e.from.y),o=t._getNewTargetNode(e.to.x,e.to.y);t.temporaryIds.nodes.push(i.id),t.temporaryIds.nodes.push(o.id),t.body.nodes[i.id]=i,t.body.nodeIndices.push(i.id),t.body.nodes[o.id]=o,t.body.nodeIndices.push(o.id),t._temporaryBindUI("onTouch",t._controlNodeTouch.bind(t)),t._temporaryBindUI("onTap",function(){}),t._temporaryBindUI("onHold",function(){}),t._temporaryBindUI("onDragStart",t._controlNodeDragStart.bind(t)),t._temporaryBindUI("onDrag",t._controlNodeDrag.bind(t)),t._temporaryBindUI("onDragEnd",t._controlNodeDragEnd.bind(t)),t._temporaryBindUI("onMouseMove",function(){}),t._temporaryBindEvent("beforeDrawing",function(t){var n=e.edgeType.findBorderPositions(t);i.selected===!1&&(i.x=n.from.x,i.y=n.from.y),o.selected===!1&&(o.x=n.to.x,o.y=n.to.y)}),t.body.emitter.emit("_redraw")}():this.showManipulatorToolbar()}},{key:"deleteSelected",value:function(){var t=this;this.editMode!==!0&&this.enableEditMode(),this._clean(),this.inMode="delete";var e=this.selectionHandler.getSelectedNodes(),i=this.selectionHandler.getSelectedEdges(),o=void 0;if(e.length>0){for(var n=0;n0&&"function"==typeof this.options.deleteEdge&&(o=this.options.deleteEdge);if("function"==typeof o){var s={nodes:e,edges:i};if(2!==o.length)throw new Error("The function for delete does not support two arguments (data, callback)");o(s,function(e){null!==e&&void 0!==e&&"delete"===t.inMode?(t.body.data.edges.getDataSet().remove(e.edges),t.body.data.nodes.getDataSet().remove(e.nodes),t.body.emitter.emit("startSimulation"),t.showManipulatorToolbar()):(t.body.emitter.emit("startSimulation"),t.showManipulatorToolbar())})}else this.body.data.edges.getDataSet().remove(i),this.body.data.nodes.getDataSet().remove(e),this.body.emitter.emit("startSimulation"),this.showManipulatorToolbar()}},{key:"_setup",value:function(){this.options.enabled===!0?(this.guiEnabled=!0,this._createWrappers(),this.editMode===!1?this._createEditButton():this.showManipulatorToolbar()):(this._removeManipulationDOM(),this.guiEnabled=!1)}},{key:"_createWrappers",value:function(){void 0===this.manipulationDiv&&(this.manipulationDiv=document.createElement("div"),this.manipulationDiv.className="vis-manipulation",this.editMode===!0?this.manipulationDiv.style.display="block":this.manipulationDiv.style.display="none",this.canvas.frame.appendChild(this.manipulationDiv)),void 0===this.editModeDiv&&(this.editModeDiv=document.createElement("div"),this.editModeDiv.className="vis-edit-mode",this.editMode===!0?this.editModeDiv.style.display="none":this.editModeDiv.style.display="block",this.canvas.frame.appendChild(this.editModeDiv)),void 0===this.closeDiv&&(this.closeDiv=document.createElement("div"),this.closeDiv.className="vis-close",this.closeDiv.style.display=this.manipulationDiv.style.display,this.canvas.frame.appendChild(this.closeDiv))}},{key:"_getNewTargetNode",value:function(t,e){var i=s.deepExtend({},this.options.controlNodeStyle);i.id="targetNode"+s.randomUUID(),i.hidden=!1,i.physics=!1,i.x=t,i.y=e;var o=this.body.functions.createNode(i);return o.shape.boundingBox={left:t,right:t,top:e,bottom:e},o}},{key:"_createEditButton",value:function(){this._clean(),this.manipulationDOM={},s.recursiveDOMDelete(this.editModeDiv);var t=this.options.locales[this.options.locale],e=this._createButton("editMode","vis-button vis-edit vis-edit-mode",t.edit||this.options.locales.en.edit);this.editModeDiv.appendChild(e),this._bindHammerToDiv(e,this.toggleEditMode.bind(this))}},{key:"_clean",value:function(){this.inMode=!1,this.guiEnabled===!0&&(s.recursiveDOMDelete(this.editModeDiv),s.recursiveDOMDelete(this.manipulationDiv),this._cleanManipulatorHammers()),this._cleanupTemporaryNodesAndEdges(),this._unbindTemporaryUIs(),this._unbindTemporaryEvents(),this.body.emitter.emit("restorePhysics")}},{key:"_cleanManipulatorHammers",value:function(){if(0!=this.manipulationHammers.length){for(var t=0;t=0;r--)if(n[r]!==this.selectedControlNode.id){s=this.body.nodes[n[r]];break}if(void 0!==s&&void 0!==this.selectedControlNode)if(s.isCluster===!0)alert(this.options.locales[this.options.locale].createEdgeError||this.options.locales.en.createEdgeError);else{var a=this.body.nodes[this.temporaryIds.nodes[0]];this.selectedControlNode.id===a.id?this._performEditEdge(s.id,o.to.id):this._performEditEdge(o.from.id,s.id)}else o.updateEdgeType(),this.body.emitter.emit("restorePhysics");this.body.emitter.emit("_redraw")}}},{key:"_handleConnect",value:function(t){if((new Date).valueOf()-this.touchTime>100){this.lastTouch=this.body.functions.getPointer(t.center),this.lastTouch.translation=s.extend({},this.body.view.translation);var e=this.lastTouch,i=this.selectionHandler.getNodeAt(e);if(void 0!==i)if(i.isCluster===!0)alert(this.options.locales[this.options.locale].createEdgeError||this.options.locales.en.createEdgeError);else{var o=this._getNewTargetNode(i.x,i.y);this.body.nodes[o.id]=o,this.body.nodeIndices.push(o.id);var n=this.body.functions.createEdge({id:"connectionEdge"+s.randomUUID(),from:i.id,to:o.id,physics:!1,smooth:{enabled:!0,type:"continuous",roundness:.5}});this.body.edges[n.id]=n,this.body.edgeIndices.push(n.id),this.temporaryIds.nodes.push(o.id),this.temporaryIds.edges.push(n.id)}this.touchTime=(new Date).valueOf()}}},{key:"_dragControlNode",value:function(t){var e=this.body.functions.getPointer(t.center);if(void 0!==this.temporaryIds.nodes[0]){var i=this.body.nodes[this.temporaryIds.nodes[0]];i.x=this.canvas._XconvertDOMtoCanvas(e.x),i.y=this.canvas._YconvertDOMtoCanvas(e.y),this.body.emitter.emit("_redraw")}else{var o=e.x-this.lastTouch.x,n=e.y-this.lastTouch.y;this.body.view.translation={x:this.lastTouch.translation.x+o,y:this.lastTouch.translation.y+n}}}},{key:"_finishConnect",value:function(t){var e=this.body.functions.getPointer(t.center),i=this.selectionHandler._pointerToPositionObject(e),o=void 0;void 0!==this.temporaryIds.edges[0]&&(o=this.body.edges[this.temporaryIds.edges[0]].fromId);for(var n=this.selectionHandler._getAllNodesOverlappingWith(i),s=void 0,r=n.length-1;r>=0;r--)if(-1===this.temporaryIds.nodes.indexOf(n[r])){s=this.body.nodes[n[r]];break}this._cleanupTemporaryNodesAndEdges(),void 0!==s&&(s.isCluster===!0?alert(this.options.locales[this.options.locale].createEdgeError||this.options.locales.en.createEdgeError):void 0!==this.body.nodes[o]&&void 0!==this.body.nodes[s.id]&&this._performAddEdge(o,s.id)),this.body.emitter.emit("_redraw")}},{key:"_performAddNode",value:function(t){var e=this,i={id:s.randomUUID(),x:t.pointer.canvas.x,y:t.pointer.canvas.y,label:"new"};if("function"==typeof this.options.addNode){if(2!==this.options.addNode.length)throw new Error("The function for add does not support two arguments (data,callback)");this.options.addNode(i,function(t){null!==t&&void 0!==t&&"addNode"===e.inMode&&(e.body.data.nodes.getDataSet().add(t),e.showManipulatorToolbar())})}else this.body.data.nodes.getDataSet().add(i),this.showManipulatorToolbar()}},{key:"_performAddEdge",value:function(t,e){var i=this,o={from:t,to:e};if("function"==typeof this.options.addEdge){if(2!==this.options.addEdge.length)throw new Error("The function for connect does not support two arguments (data,callback)");this.options.addEdge(o,function(t){null!==t&&void 0!==t&&"addEdge"===i.inMode&&(i.body.data.edges.getDataSet().add(t),i.selectionHandler.unselectAll(),i.showManipulatorToolbar())})}else this.body.data.edges.getDataSet().add(o),this.selectionHandler.unselectAll(),this.showManipulatorToolbar()}},{key:"_performEditEdge",value:function(t,e){var i=this,o={id:this.edgeBeingEditedId,from:t,to:e};if("function"==typeof this.options.editEdge){if(2!==this.options.editEdge.length)throw new Error("The function for edit does not support two arguments (data, callback)");this.options.editEdge(o,function(t){null===t||void 0===t||"editEdge"!==i.inMode?(i.body.edges[o.id].updateEdgeType(),i.body.emitter.emit("_redraw")):(i.body.data.edges.getDataSet().update(t),i.selectionHandler.unselectAll(),i.showManipulatorToolbar())})}else this.body.data.edges.getDataSet().update(o),this.selectionHandler.unselectAll(),this.showManipulatorToolbar()}}]),t}();e["default"]=h},function(t,e){Object.defineProperty(e,"__esModule",{value:!0});var i="string",o="boolean",n="number",s="array",r="object",a="dom",h="any",d={configure:{enabled:{"boolean":o},filter:{"boolean":o,string:i,array:s,"function":"function"},container:{dom:a},showButton:{"boolean":o},__type__:{object:r,"boolean":o,string:i,array:s,"function":"function"}},edges:{arrows:{to:{enabled:{"boolean":o},scaleFactor:{number:n},__type__:{object:r,"boolean":o}},middle:{enabled:{"boolean":o},scaleFactor:{number:n},__type__:{object:r,"boolean":o}},from:{enabled:{"boolean":o},scaleFactor:{number:n},__type__:{object:r,"boolean":o}},__type__:{string:["from","to","middle"],object:r}},arrowStrikethrough:{"boolean":o},color:{color:{string:i},highlight:{string:i},hover:{string:i},inherit:{string:["from","to","both"],"boolean":o},opacity:{number:n},__type__:{object:r,string:i}},dashes:{"boolean":o,array:s},font:{color:{string:i},size:{number:n},face:{string:i},background:{string:i},strokeWidth:{number:n},strokeColor:{string:i},align:{string:["horizontal","top","middle","bottom"]},__type__:{object:r,string:i}},hidden:{"boolean":o},hoverWidth:{"function":"function",number:n},label:{string:i,undefined:"undefined"},labelHighlightBold:{"boolean":o},length:{number:n,undefined:"undefined"},physics:{"boolean":o},scaling:{min:{number:n},max:{number:n},label:{enabled:{"boolean":o},min:{number:n},max:{number:n},maxVisible:{number:n},drawThreshold:{number:n},__type__:{object:r,"boolean":o}},customScalingFunction:{"function":"function"},__type__:{object:r}},selectionWidth:{"function":"function",number:n},selfReferenceSize:{number:n},shadow:{enabled:{"boolean":o},color:{string:i},size:{number:n},x:{number:n},y:{number:n},__type__:{object:r,"boolean":o}},smooth:{enabled:{"boolean":o},type:{string:["dynamic","continuous","discrete","diagonalCross","straightCross","horizontal","vertical","curvedCW","curvedCCW","cubicBezier"]},roundness:{number:n},forceDirection:{string:["horizontal","vertical","none"],"boolean":o},__type__:{object:r,"boolean":o}},title:{string:i,undefined:"undefined"},width:{number:n},value:{number:n,undefined:"undefined"},__type__:{object:r}},groups:{useDefaultGroups:{"boolean":o},__any__:"get from nodes, will be overwritten below",__type__:{object:r}},interaction:{dragNodes:{"boolean":o},dragView:{"boolean":o},hideEdgesOnDrag:{"boolean":o},hideNodesOnDrag:{"boolean":o},hover:{"boolean":o},keyboard:{enabled:{"boolean":o},speed:{x:{number:n},y:{number:n},zoom:{number:n},__type__:{object:r}},bindToWindow:{"boolean":o},__type__:{object:r,"boolean":o}},multiselect:{"boolean":o},navigationButtons:{"boolean":o},selectable:{"boolean":o},selectConnectedEdges:{"boolean":o},hoverConnectedEdges:{"boolean":o},tooltipDelay:{number:n},zoomView:{"boolean":o},__type__:{object:r}},layout:{randomSeed:{undefined:"undefined",number:n},improvedLayout:{"boolean":o},hierarchical:{enabled:{"boolean":o},levelSeparation:{number:n},nodeSpacing:{number:n},treeSpacing:{number:n},blockShifting:{"boolean":o},edgeMinimization:{"boolean":o},parentCentralization:{"boolean":o},direction:{string:["UD","DU","LR","RL"]},sortMethod:{string:["hubsize","directed"]},__type__:{object:r,"boolean":o}},__type__:{object:r}},manipulation:{enabled:{"boolean":o},initiallyActive:{"boolean":o},addNode:{"boolean":o,"function":"function"},addEdge:{"boolean":o,"function":"function"},editNode:{"function":"function"},editEdge:{"boolean":o,"function":"function"},deleteNode:{"boolean":o,"function":"function"},deleteEdge:{"boolean":o,"function":"function"},controlNodeStyle:"get from nodes, will be overwritten below",__type__:{object:r,"boolean":o}},nodes:{borderWidth:{number:n},borderWidthSelected:{number:n,undefined:"undefined"},brokenImage:{string:i,undefined:"undefined"},color:{border:{string:i},background:{string:i},highlight:{border:{string:i},background:{string:i},__type__:{object:r,string:i}},hover:{border:{string:i},background:{string:i},__type__:{object:r,string:i}},__type__:{object:r,string:i}},fixed:{x:{"boolean":o},y:{"boolean":o},__type__:{object:r,"boolean":o}},font:{align:{string:i},color:{string:i},size:{number:n},face:{string:i},background:{string:i},strokeWidth:{number:n},strokeColor:{string:i},__type__:{object:r,string:i}},group:{string:i,number:n,undefined:"undefined"},hidden:{"boolean":o},icon:{face:{string:i},code:{string:i},size:{number:n},color:{string:i},__type__:{object:r}},id:{string:i,number:n},image:{string:i,undefined:"undefined"},label:{string:i,undefined:"undefined"},labelHighlightBold:{"boolean":o},level:{number:n,undefined:"undefined"},mass:{number:n},physics:{"boolean":o},scaling:{min:{number:n},max:{number:n},label:{enabled:{"boolean":o},min:{number:n},max:{number:n},maxVisible:{number:n},drawThreshold:{number:n},__type__:{object:r,"boolean":o}},customScalingFunction:{"function":"function"},__type__:{object:r}},shadow:{enabled:{"boolean":o},color:{string:i},size:{number:n},x:{number:n},y:{number:n},__type__:{object:r,"boolean":o}},shape:{string:["ellipse","circle","database","box","text","image","circularImage","diamond","dot","star","triangle","triangleDown","square","icon"]},shapeProperties:{borderDashes:{"boolean":o,array:s},borderRadius:{number:n},interpolation:{"boolean":o},useImageSize:{"boolean":o},useBorderWithImage:{"boolean":o},__type__:{object:r}},size:{number:n},title:{string:i,undefined:"undefined"},value:{number:n,undefined:"undefined"},x:{number:n},y:{number:n},__type__:{object:r}},physics:{enabled:{"boolean":o},barnesHut:{gravitationalConstant:{number:n},centralGravity:{number:n},springLength:{number:n},springConstant:{number:n},damping:{number:n},avoidOverlap:{number:n},__type__:{object:r}},forceAtlas2Based:{gravitationalConstant:{number:n},centralGravity:{number:n},springLength:{number:n},springConstant:{number:n},damping:{number:n},avoidOverlap:{number:n},__type__:{object:r}},repulsion:{centralGravity:{number:n},springLength:{number:n},springConstant:{number:n},nodeDistance:{number:n},damping:{number:n},__type__:{object:r}},hierarchicalRepulsion:{centralGravity:{number:n},springLength:{number:n},springConstant:{number:n},nodeDistance:{number:n},damping:{number:n},__type__:{object:r}},maxVelocity:{number:n},minVelocity:{number:n},solver:{string:["barnesHut","repulsion","hierarchicalRepulsion","forceAtlas2Based"]},stabilization:{enabled:{"boolean":o},iterations:{number:n},updateInterval:{number:n},onlyDynamicEdges:{"boolean":o},fit:{"boolean":o},__type__:{object:r,"boolean":o}},timestep:{number:n},adaptiveTimestep:{"boolean":o},__type__:{object:r,"boolean":o}},autoResize:{"boolean":o},clickToUse:{"boolean":o},locale:{string:i},locales:{__any__:{any:h},__type__:{object:r}},height:{string:i},width:{string:i},__type__:{object:r}};d.groups.__any__=d.nodes,d.manipulation.controlNodeStyle=d.nodes;var l={nodes:{borderWidth:[1,0,10,1],borderWidthSelected:[2,0,10,1],color:{border:["color","#2B7CE9"],background:["color","#97C2FC"],highlight:{border:["color","#2B7CE9"],background:["color","#D2E5FF"]},hover:{border:["color","#2B7CE9"],background:["color","#D2E5FF"]}},fixed:{x:!1,y:!1},font:{color:["color","#343434"],size:[14,0,100,1],face:["arial","verdana","tahoma"],background:["color","none"],strokeWidth:[0,0,50,1],strokeColor:["color","#ffffff"]},hidden:!1,labelHighlightBold:!0,physics:!0,scaling:{min:[10,0,200,1],max:[30,0,200,1],label:{enabled:!1,min:[14,0,200,1],max:[30,0,200,1],maxVisible:[30,0,200,1],drawThreshold:[5,0,20,1]}},shadow:{enabled:!1,color:"rgba(0,0,0,0.5)",size:[10,0,20,1],x:[5,-30,30,1],y:[5,-30,30,1]},shape:["ellipse","box","circle","database","diamond","dot","square","star","text","triangle","triangleDown"],shapeProperties:{borderDashes:!1,borderRadius:[6,0,20,1],interpolation:!0,useImageSize:!1},size:[25,0,200,1]},edges:{arrows:{to:{enabled:!1,scaleFactor:[1,0,3,.05]},middle:{enabled:!1,scaleFactor:[1,0,3,.05]},from:{enabled:!1,scaleFactor:[1,0,3,.05]}},arrowStrikethrough:!0,color:{color:["color","#848484"],highlight:["color","#848484"],hover:["color","#848484"],inherit:["from","to","both",!0,!1],opacity:[1,0,1,.05]},dashes:!1,font:{color:["color","#343434"],size:[14,0,100,1],face:["arial","verdana","tahoma"],background:["color","none"],strokeWidth:[2,0,50,1],strokeColor:["color","#ffffff"],align:["horizontal","top","middle","bottom"]},hidden:!1,hoverWidth:[1.5,0,5,.1],labelHighlightBold:!0,physics:!0,scaling:{min:[1,0,100,1],max:[15,0,100,1],label:{enabled:!0,min:[14,0,200,1],max:[30,0,200,1],maxVisible:[30,0,200,1],drawThreshold:[5,0,20,1]}},selectionWidth:[1.5,0,5,.1],selfReferenceSize:[20,0,200,1],shadow:{enabled:!1,color:"rgba(0,0,0,0.5)",size:[10,0,20,1],x:[5,-30,30,1],y:[5,-30,30,1]},smooth:{enabled:!0,type:["dynamic","continuous","discrete","diagonalCross","straightCross","horizontal","vertical","curvedCW","curvedCCW","cubicBezier"],forceDirection:["horizontal","vertical","none"],roundness:[.5,0,1,.05]},width:[1,0,30,1]},layout:{hierarchical:{enabled:!1,levelSeparation:[150,20,500,5],nodeSpacing:[100,20,500,5],treeSpacing:[200,20,500,5],blockShifting:!0,edgeMinimization:!0,parentCentralization:!0,direction:["UD","DU","LR","RL"],sortMethod:["hubsize","directed"]}},interaction:{dragNodes:!0,dragView:!0,hideEdgesOnDrag:!1,hideNodesOnDrag:!1,hover:!1,keyboard:{enabled:!1,speed:{x:[10,0,40,1],y:[10,0,40,1],zoom:[.02,0,.1,.005]},bindToWindow:!0},multiselect:!1,navigationButtons:!1,selectable:!0,selectConnectedEdges:!0,hoverConnectedEdges:!0,tooltipDelay:[300,0,1e3,25],zoomView:!0},manipulation:{enabled:!1,initiallyActive:!1},physics:{enabled:!0,barnesHut:{gravitationalConstant:[-2e3,-3e4,0,50],centralGravity:[.3,0,10,.05],springLength:[95,0,500,5],springConstant:[.04,0,1.2,.005],damping:[.09,0,1,.01],avoidOverlap:[0,0,1,.01]},forceAtlas2Based:{gravitationalConstant:[-50,-500,0,1],centralGravity:[.01,0,1,.005],springLength:[95,0,500,5],springConstant:[.08,0,1.2,.005],damping:[.4,0,1,.01],avoidOverlap:[0,0,1,.01]},repulsion:{centralGravity:[.2,0,10,.05],springLength:[200,0,500,5],springConstant:[.05,0,1.2,.005],nodeDistance:[100,0,500,5],damping:[.09,0,1,.01]},hierarchicalRepulsion:{centralGravity:[.2,0,10,.05],springLength:[100,0,500,5],springConstant:[.01,0,1.2,.005],nodeDistance:[120,0,500,5],damping:[.09,0,1,.01]},maxVelocity:[50,0,150,1],minVelocity:[.1,.01,.5,.01],solver:["barnesHut","forceAtlas2Based","repulsion","hierarchicalRepulsion"],timestep:[.5,.01,1,.01]},global:{locale:["en","nl"]}};e.allOptions=d,e.configureOptions=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s=function(){function t(t,e){var i=[],o=!0,n=!1,s=void 0;try{for(var r,a=t[Symbol.iterator]();!(o=(r=a.next()).done)&&(i.push(r.value),!e||i.length!==e);o=!0);}catch(h){n=!0,s=h}finally{try{!o&&a["return"]&&a["return"]()}finally{if(n)throw s}}return i}return function(e,i){if(Array.isArray(e))return e;if(Symbol.iterator in Object(e))return t(e,i);throw new TypeError("Invalid attempt to destructure non-iterable instance")}}(),r=function(){function t(t,e){for(var i=0;in&&h>a;){a+=1;var v=this._getHighestEnergyNode(i),g=s(v,4);for(c=g[0],l=g[1],u=g[2],p=g[3],f=l,m=0;f>r&&d>m;){m+=1,this._moveNode(c,u,p);var y=this._getEnergy(c),b=s(y,3);f=b[0],u=b[1],p=b[2]}}}},{key:"_getHighestEnergyNode",value:function(t){for(var e=this.body.nodeIndices,i=this.body.nodes,o=0,n=e[0],r=0,a=0,h=0;ho&&(o=u,n=d,r=p,a=f)}}return[n,o,r,a]}},{key:"_getEnergy",value:function(t){for(var e=this.body.nodeIndices,i=this.body.nodes,o=i[t].x,n=i[t].y,s=0,r=0,a=0;al;l++)for(var c=0;d-1>c;c++)for(var u=c+1;d>u;u++)o[e[c]][e[u]]=Math.min(o[e[c]][e[u]],o[e[c]][e[l]]+o[e[l]][e[u]]),o[e[u]][e[c]]=o[e[c]][e[u]];return o}}]),t}();e["default"]=n},function(t,e){"undefined"!=typeof CanvasRenderingContext2D&&(CanvasRenderingContext2D.prototype.circle=function(t,e,i){this.beginPath(),this.arc(t,e,i,0,2*Math.PI,!1),this.closePath()},CanvasRenderingContext2D.prototype.square=function(t,e,i){this.beginPath(),this.rect(t-i,e-i,2*i,2*i),this.closePath()},CanvasRenderingContext2D.prototype.triangle=function(t,e,i){this.beginPath(),i*=1.15,e+=.275*i;var o=2*i,n=o/2,s=Math.sqrt(3)/6*o,r=Math.sqrt(o*o-n*n);this.moveTo(t,e-(r-s)),this.lineTo(t+n,e+s),this.lineTo(t-n,e+s),this.lineTo(t,e-(r-s)),this.closePath()},CanvasRenderingContext2D.prototype.triangleDown=function(t,e,i){this.beginPath(),i*=1.15,e-=.275*i;var o=2*i,n=o/2,s=Math.sqrt(3)/6*o,r=Math.sqrt(o*o-n*n);this.moveTo(t,e+(r-s)), +this.lineTo(t+n,e-s),this.lineTo(t-n,e-s),this.lineTo(t,e+(r-s)),this.closePath()},CanvasRenderingContext2D.prototype.star=function(t,e,i){this.beginPath(),i*=.82,e+=.1*i;for(var o=0;10>o;o++){var n=o%2===0?1.3*i:.5*i;this.lineTo(t+n*Math.sin(2*o*Math.PI/10),e-n*Math.cos(2*o*Math.PI/10))}this.closePath()},CanvasRenderingContext2D.prototype.diamond=function(t,e,i){this.beginPath(),this.lineTo(t,e+i),this.lineTo(t+i,e),this.lineTo(t,e-i),this.lineTo(t-i,e),this.closePath()},CanvasRenderingContext2D.prototype.roundRect=function(t,e,i,o,n){var s=Math.PI/180;0>i-2*n&&(n=i/2),0>o-2*n&&(n=o/2),this.beginPath(),this.moveTo(t+n,e),this.lineTo(t+i-n,e),this.arc(t+i-n,e+n,n,270*s,360*s,!1),this.lineTo(t+i,e+o-n),this.arc(t+i-n,e+o-n,n,0,90*s,!1),this.lineTo(t+n,e+o),this.arc(t+n,e+o-n,n,90*s,180*s,!1),this.lineTo(t,e+n),this.arc(t+n,e+n,n,180*s,270*s,!1),this.closePath()},CanvasRenderingContext2D.prototype.ellipse=function(t,e,i,o){var n=.5522848,s=i/2*n,r=o/2*n,a=t+i,h=e+o,d=t+i/2,l=e+o/2;this.beginPath(),this.moveTo(t,l),this.bezierCurveTo(t,l-r,d-s,e,d,e),this.bezierCurveTo(d+s,e,a,l-r,a,l),this.bezierCurveTo(a,l+r,d+s,h,d,h),this.bezierCurveTo(d-s,h,t,l+r,t,l),this.closePath()},CanvasRenderingContext2D.prototype.database=function(t,e,i,o){var n=1/3,s=i,r=o*n,a=.5522848,h=s/2*a,d=r/2*a,l=t+s,c=e+r,u=t+s/2,p=e+r/2,f=e+(o-r/2),m=e+o;this.beginPath(),this.moveTo(l,p),this.bezierCurveTo(l,p+d,u+h,c,u,c),this.bezierCurveTo(u-h,c,t,p+d,t,p),this.bezierCurveTo(t,p-d,u-h,e,u,e),this.bezierCurveTo(u+h,e,l,p-d,l,p),this.lineTo(l,f),this.bezierCurveTo(l,f+d,u+h,m,u,m),this.bezierCurveTo(u-h,m,t,f+d,t,f),this.lineTo(t,p)},CanvasRenderingContext2D.prototype.arrow=function(t,e,i,o){var n=t-o*Math.cos(i),s=e-o*Math.sin(i),r=t-.9*o*Math.cos(i),a=e-.9*o*Math.sin(i),h=n+o/3*Math.cos(i+.5*Math.PI),d=s+o/3*Math.sin(i+.5*Math.PI),l=n+o/3*Math.cos(i-.5*Math.PI),c=s+o/3*Math.sin(i-.5*Math.PI);this.beginPath(),this.moveTo(t,e),this.lineTo(h,d),this.lineTo(r,a),this.lineTo(l,c),this.closePath()},CanvasRenderingContext2D.prototype.dashedLine=function(t,e,i,o,n){this.beginPath(),this.moveTo(t,e);for(var s=n.length,r=i-t,a=o-e,h=a/r,d=Math.sqrt(r*r+a*a),l=0,c=!0,u=0,p=n[0];d>=.1;)p=n[l++%s],p>d&&(p=d),u=Math.sqrt(p*p/(1+h*h)),u=0>r?-u:u,t+=u,e+=h*u,c===!0?this.lineTo(t,e):this.moveTo(t,e),d-=p,c=!c})},function(t,e){function i(t){return P=t,p()}function o(){I=0,N=P.charAt(0)}function n(){I++,N=P.charAt(I)}function s(){return P.charAt(I+1)}function r(t){return L.test(t)}function a(t,e){if(t||(t={}),e)for(var i in e)e.hasOwnProperty(i)&&(t[i]=e[i]);return t}function h(t,e,i){for(var o=e.split("."),n=t;o.length;){var s=o.shift();o.length?(n[s]||(n[s]={}),n=n[s]):n[s]=i}}function d(t,e){for(var i,o,n=null,s=[t],r=t;r.parent;)s.push(r.parent),r=r.parent;if(r.nodes)for(i=0,o=r.nodes.length;o>i;i++)if(e.id===r.nodes[i].id){n=r.nodes[i];break}for(n||(n={id:e.id},t.node&&(n.attr=a(n.attr,t.node))),i=s.length-1;i>=0;i--){var h=s[i];h.nodes||(h.nodes=[]),-1===h.nodes.indexOf(n)&&h.nodes.push(n)}e.attr&&(n.attr=a(n.attr,e.attr))}function l(t,e){if(t.edges||(t.edges=[]),t.edges.push(e),t.edge){var i=a({},t.edge);e.attr=a(i,e.attr)}}function c(t,e,i,o,n){var s={from:e,to:i,type:o};return t.edge&&(s.attr=a({},t.edge)),s.attr=a(s.attr||{},n),s}function u(){for(z=T.NULL,R="";" "===N||" "===N||"\n"===N||"\r"===N;)n();do{var t=!1;if("#"===N){for(var e=I-1;" "===P.charAt(e)||" "===P.charAt(e);)e--;if("\n"===P.charAt(e)||""===P.charAt(e)){for(;""!=N&&"\n"!=N;)n();t=!0}}if("/"===N&&"/"===s()){for(;""!=N&&"\n"!=N;)n();t=!0}if("/"===N&&"*"===s()){for(;""!=N;){if("*"===N&&"/"===s()){n(),n();break}n()}t=!0}for(;" "===N||" "===N||"\n"===N||"\r"===N;)n()}while(t);if(""===N)return void(z=T.DELIMITER);var i=N+s();if(E[i])return z=T.DELIMITER,R=i,n(),void n();if(E[N])return z=T.DELIMITER,R=N,void n();if(r(N)||"-"===N){for(R+=N,n();r(N);)R+=N,n();return"false"===R?R=!1:"true"===R?R=!0:isNaN(Number(R))||(R=Number(R)),void(z=T.IDENTIFIER)}if('"'===N){for(n();""!=N&&('"'!=N||'"'===N&&'"'===s());)R+=N,'"'===N&&n(),n();if('"'!=N)throw _('End of string " expected');return n(),void(z=T.IDENTIFIER)}for(z=T.UNKNOWN;""!=N;)R+=N,n();throw new SyntaxError('Syntax error in part "'+x(R,30)+'"')}function p(){var t={};if(o(),u(),"strict"===R&&(t.strict=!0,u()),"graph"!==R&&"digraph"!==R||(t.type=R,u()),z===T.IDENTIFIER&&(t.id=R,u()),"{"!=R)throw _("Angle bracket { expected");if(u(),f(t),"}"!=R)throw _("Angle bracket } expected");if(u(),""!==R)throw _("End of file expected");return u(),delete t.node,delete t.edge,delete t.graph,t}function f(t){for(;""!==R&&"}"!=R;)m(t),";"===R&&u()}function m(t){var e=v(t);if(e)return void b(t,e);var i=g(t);if(!i){if(z!=T.IDENTIFIER)throw _("Identifier expected");var o=R;if(u(),"="===R){if(u(),z!=T.IDENTIFIER)throw _("Identifier expected");t[o]=R,u()}else y(t,o)}}function v(t){var e=null;if("subgraph"===R&&(e={},e.type="subgraph",u(),z===T.IDENTIFIER&&(e.id=R,u())),"{"===R){if(u(),e||(e={}),e.parent=t,e.node=t.node,e.edge=t.edge,e.graph=t.graph,f(e),"}"!=R)throw _("Angle bracket } expected");u(),delete e.node,delete e.edge,delete e.graph,delete e.parent,t.subgraphs||(t.subgraphs=[]),t.subgraphs.push(e)}return e}function g(t){return"node"===R?(u(),t.node=w(),"node"):"edge"===R?(u(),t.edge=w(),"edge"):"graph"===R?(u(),t.graph=w(),"graph"):null}function y(t,e){var i={id:e},o=w();o&&(i.attr=o),d(t,i),b(t,e)}function b(t,e){for(;"->"===R||"--"===R;){var i,o=R;u();var n=v(t);if(n)i=n;else{if(z!=T.IDENTIFIER)throw _("Identifier or subgraph expected");i=R,d(t,{id:i}),u()}var s=w(),r=c(t,e,i,o,s);l(t,r),e=i}}function w(){for(var t=null;"["===R;){for(u(),t={};""!==R&&"]"!=R;){if(z!=T.IDENTIFIER)throw _("Attribute name expected");var e=R;if(u(),"="!=R)throw _("Equal sign = expected");if(u(),z!=T.IDENTIFIER)throw _("Attribute value expected");var i=R;h(t,e,i),u(),","==R&&u()}if("]"!=R)throw _("Bracket ] expected");u()}return t}function _(t){return new SyntaxError(t+', got "'+x(R,30)+'" (char '+I+")")}function x(t,e){return t.length<=e?t:t.substr(0,27)+"..."}function k(t,e,i){Array.isArray(t)?t.forEach(function(t){Array.isArray(e)?e.forEach(function(e){i(t,e)}):i(t,e)}):Array.isArray(e)?e.forEach(function(e){i(t,e)}):i(t,e)}function O(t,e,i){for(var o=e.split("."),n=o.pop(),s=t,r=0;r":!0,"--":!0},P="",I=0,N="",R="",z=T.NULL,L=/[a-zA-Z_0-9.:#]/;e.parseDOT=i,e.DOTToGraph=D},function(t,e){function i(t,e){var i=[],o=[],n={edges:{inheritColor:!1},nodes:{fixed:!1,parseColor:!1}};void 0!==e&&(void 0!==e.fixed&&(n.nodes.fixed=e.fixed),void 0!==e.parseColor&&(n.nodes.parseColor=e.parseColor),void 0!==e.inheritColor&&(n.edges.inheritColor=e.inheritColor));for(var s=t.edges,r=t.nodes,a=0;a val accum = sc.accumulator(0) - * accum: spark.Accumulator[Int] = 0 + * accum: org.apache.spark.Accumulator[Int] = 0 * * scala> sc.parallelize(Array(1, 2, 3, 4)).foreach(x => accum += x) * ... @@ -58,7 +49,8 @@ import org.apache.spark.storage.{BlockId, BlockStatus} * @param name human-readable name associated with this accumulator * @param countFailedValues whether to accumulate values from failed tasks * @tparam T result type - */ +*/ +@deprecated("use AccumulatorV2", "2.0.0") class Accumulator[T] private[spark] ( // SI-8813: This must explicitly be a private val, or else scala 2.11 doesn't compile @transient private val initialValue: T, @@ -75,6 +67,7 @@ class Accumulator[T] private[spark] ( * * @tparam T type of value to accumulate */ +@deprecated("use AccumulatorV2", "2.0.0") trait AccumulatorParam[T] extends AccumulableParam[T, T] { def addAccumulator(t1: T, t2: T): T = { addInPlace(t1, t2) @@ -82,6 +75,7 @@ trait AccumulatorParam[T] extends AccumulableParam[T, T] { } +@deprecated("use AccumulatorV2", "2.0.0") object AccumulatorParam { // The following implicit objects were in SparkContext before 1.2 and users had to @@ -89,21 +83,25 @@ object AccumulatorParam { // them automatically. However, as there are duplicate codes in SparkContext for backward // compatibility, please update them accordingly if you modify the following implicit objects. + @deprecated("use AccumulatorV2", "2.0.0") implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 def zero(initialValue: Double): Double = 0.0 } + @deprecated("use AccumulatorV2", "2.0.0") implicit object IntAccumulatorParam extends AccumulatorParam[Int] { def addInPlace(t1: Int, t2: Int): Int = t1 + t2 def zero(initialValue: Int): Int = 0 } + @deprecated("use AccumulatorV2", "2.0.0") implicit object LongAccumulatorParam extends AccumulatorParam[Long] { def addInPlace(t1: Long, t2: Long): Long = t1 + t2 def zero(initialValue: Long): Long = 0L } + @deprecated("use AccumulatorV2", "2.0.0") implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { def addInPlace(t1: Float, t2: Float): Float = t1 + t2 def zero(initialValue: Float): Float = 0f @@ -112,20 +110,9 @@ object AccumulatorParam { // Note: when merging values, this param just adopts the newer value. This is used only // internally for things that shouldn't really be accumulated across tasks, like input // read method, which should be the same across all tasks in the same stage. + @deprecated("use AccumulatorV2", "2.0.0") private[spark] object StringAccumulatorParam extends AccumulatorParam[String] { def addInPlace(t1: String, t2: String): String = t2 def zero(initialValue: String): String = "" } - - // Note: this is expensive as it makes a copy of the list every time the caller adds an item. - // A better way to use this is to first accumulate the values yourself then them all at once. - private[spark] class ListAccumulatorParam[T] extends AccumulatorParam[Seq[T]] { - def addInPlace(t1: Seq[T], t2: Seq[T]): Seq[T] = t1 ++ t2 - def zero(initialValue: Seq[T]): Seq[T] = Seq.empty[T] - } - - // For the internal metric that records what blocks are updated in a particular task - private[spark] object UpdatedBlockStatusesAccumulatorParam - extends ListAccumulatorParam[(BlockId, BlockStatus)] - } diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 63a00a84af3c..5678d790e9e7 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData} -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, ThreadUtils, Utils} /** * Classes that represent cleaning tasks. @@ -144,7 +144,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { registerForCleanup(rdd, CleanRDD(rdd.id)) } - def registerAccumulatorForCleanup(a: NewAccumulator[_, _]): Unit = { + def registerAccumulatorForCleanup(a: AccumulatorV2[_, _]): Unit = { registerForCleanup(a, CleanAccum(a.id)) } @@ -212,7 +212,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } } - /** Perform shuffle cleanup, asynchronously. */ + /** Perform shuffle cleanup. */ def doCleanupShuffle(shuffleId: Int, blocking: Boolean): Unit = { try { logDebug("Cleaning shuffle " + shuffleId) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 9eac05fdf9f3..73495a8d7d7f 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -26,7 +26,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} +import org.apache.spark.util._ /** * A heartbeat from executors to the driver. This is a shared message used by several internal @@ -35,7 +35,7 @@ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} */ private[spark] case class Heartbeat( executorId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], // taskId -> accumulator updates + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], // taskId -> accumulator updates blockManagerId: BlockManagerId) /** diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 3a5caa3510eb..6bd950205fad 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -18,13 +18,15 @@ package org.apache.spark import java.io._ -import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue, ThreadPoolExecutor} import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import scala.reflect.ClassTag +import scala.util.control.NonFatal +import org.apache.spark.broadcast.{Broadcast, BroadcastManager} import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.MapStatus @@ -37,31 +39,18 @@ private[spark] case class GetMapOutputStatuses(shuffleId: Int) extends MapOutputTrackerMessage private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage +private[spark] case class GetMapOutputMessage(shuffleId: Int, context: RpcCallContext) + /** RpcEndpoint class for MapOutputTrackerMaster */ private[spark] class MapOutputTrackerMasterEndpoint( override val rpcEnv: RpcEnv, tracker: MapOutputTrackerMaster, conf: SparkConf) extends RpcEndpoint with Logging { - val maxRpcMessageSize = RpcUtils.maxMessageSizeBytes(conf) override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case GetMapOutputStatuses(shuffleId: Int) => val hostPort = context.senderAddress.hostPort logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + hostPort) - val mapOutputStatuses = tracker.getSerializedMapOutputStatuses(shuffleId) - val serializedSize = mapOutputStatuses.length - if (serializedSize > maxRpcMessageSize) { - - val msg = s"Map output statuses were $serializedSize bytes which " + - s"exceeds spark.rpc.message.maxSize ($maxRpcMessageSize bytes)." - - /* For SPARK-1244 we'll opt for just logging an error and then sending it to the sender. - * A bigger refactoring (SPARK-1239) will ultimately remove this entire code path. */ - val exception = new SparkException(msg) - logError(msg, exception) - context.sendFailure(exception) - } else { - context.reply(mapOutputStatuses) - } + val mapOutputStatuses = tracker.post(new GetMapOutputMessage(shuffleId, context)) case StopMapOutputTracker => logInfo("MapOutputTrackerMasterEndpoint stopped!") @@ -270,12 +259,17 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging /** * MapOutputTracker for the driver. */ -private[spark] class MapOutputTrackerMaster(conf: SparkConf) +private[spark] class MapOutputTrackerMaster(conf: SparkConf, + broadcastManager: BroadcastManager, isLocal: Boolean) extends MapOutputTracker(conf) { /** Cache a serialized version of the output statuses for each shuffle to send them out faster */ private var cacheEpoch = epoch + // The size at which we use Broadcast to send the map output statuses to the executors + private val minSizeForBroadcast = + conf.getSizeAsBytes("spark.shuffle.mapOutput.minSizeForBroadcast", "512k").toInt + /** Whether to compute locality preferences for reduce tasks */ private val shuffleLocalityEnabled = conf.getBoolean("spark.shuffle.reduceLocality.enabled", true) @@ -296,10 +290,86 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) protected val mapStatuses = new ConcurrentHashMap[Int, Array[MapStatus]]().asScala private val cachedSerializedStatuses = new ConcurrentHashMap[Int, Array[Byte]]().asScala + private val maxRpcMessageSize = RpcUtils.maxMessageSizeBytes(conf) + + // Kept in sync with cachedSerializedStatuses explicitly + // This is required so that the Broadcast variable remains in scope until we remove + // the shuffleId explicitly or implicitly. + private val cachedSerializedBroadcast = new HashMap[Int, Broadcast[Array[Byte]]]() + + // This is to prevent multiple serializations of the same shuffle - which happens when + // there is a request storm when shuffle start. + private val shuffleIdLocks = new ConcurrentHashMap[Int, AnyRef]() + + // requests for map output statuses + private val mapOutputRequests = new LinkedBlockingQueue[GetMapOutputMessage] + + // Thread pool used for handling map output status requests. This is a separate thread pool + // to ensure we don't block the normal dispatcher threads. + private val threadpool: ThreadPoolExecutor = { + val numThreads = conf.getInt("spark.shuffle.mapOutput.dispatcher.numThreads", 8) + val pool = ThreadUtils.newDaemonFixedThreadPool(numThreads, "map-output-dispatcher") + for (i <- 0 until numThreads) { + pool.execute(new MessageLoop) + } + pool + } + + // Make sure that that we aren't going to exceed the max RPC message size by making sure + // we use broadcast to send large map output statuses. + if (minSizeForBroadcast > maxRpcMessageSize) { + val msg = s"spark.shuffle.mapOutput.minSizeForBroadcast ($minSizeForBroadcast bytes) must " + + s"be <= spark.rpc.message.maxSize ($maxRpcMessageSize bytes) to prevent sending an rpc " + + "message that is to large." + logError(msg) + throw new IllegalArgumentException(msg) + } + + def post(message: GetMapOutputMessage): Unit = { + mapOutputRequests.offer(message) + } + + /** Message loop used for dispatching messages. */ + private class MessageLoop extends Runnable { + override def run(): Unit = { + try { + while (true) { + try { + val data = mapOutputRequests.take() + if (data == PoisonPill) { + // Put PoisonPill back so that other MessageLoops can see it. + mapOutputRequests.offer(PoisonPill) + return + } + val context = data.context + val shuffleId = data.shuffleId + val hostPort = context.senderAddress.hostPort + logDebug("Handling request to send map output locations for shuffle " + shuffleId + + " to " + hostPort) + val mapOutputStatuses = getSerializedMapOutputStatuses(shuffleId) + context.reply(mapOutputStatuses) + } catch { + case NonFatal(e) => logError(e.getMessage, e) + } + } + } catch { + case ie: InterruptedException => // exit + } + } + } + + /** A poison endpoint that indicates MessageLoop should exit its message loop. */ + private val PoisonPill = new GetMapOutputMessage(-99, null) + + // Exposed for testing + private[spark] def getNumCachedSerializedBroadcast = cachedSerializedBroadcast.size + def registerShuffle(shuffleId: Int, numMaps: Int) { if (mapStatuses.put(shuffleId, new Array[MapStatus](numMaps)).isDefined) { throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice") } + // add in advance + shuffleIdLocks.putIfAbsent(shuffleId, new Object()) } def registerMapOutput(shuffleId: Int, mapId: Int, status: MapStatus) { @@ -337,6 +407,8 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) override def unregisterShuffle(shuffleId: Int) { mapStatuses.remove(shuffleId) cachedSerializedStatuses.remove(shuffleId) + cachedSerializedBroadcast.remove(shuffleId).foreach(v => removeBroadcast(v)) + shuffleIdLocks.remove(shuffleId) } /** Check if the given shuffle is being tracked */ @@ -428,40 +500,89 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } } + private def removeBroadcast(bcast: Broadcast[_]): Unit = { + if (null != bcast) { + broadcastManager.unbroadcast(bcast.id, + removeFromDriver = true, blocking = false) + } + } + + private def clearCachedBroadcast(): Unit = { + for (cached <- cachedSerializedBroadcast) removeBroadcast(cached._2) + cachedSerializedBroadcast.clear() + } + def getSerializedMapOutputStatuses(shuffleId: Int): Array[Byte] = { var statuses: Array[MapStatus] = null + var retBytes: Array[Byte] = null var epochGotten: Long = -1 - epochLock.synchronized { - if (epoch > cacheEpoch) { - cachedSerializedStatuses.clear() - cacheEpoch = epoch - } - cachedSerializedStatuses.get(shuffleId) match { - case Some(bytes) => - return bytes - case None => - statuses = mapStatuses.getOrElse(shuffleId, Array[MapStatus]()) - epochGotten = epoch + + // Check to see if we have a cached version, returns true if it does + // and has side effect of setting retBytes. If not returns false + // with side effect of setting statuses + def checkCachedStatuses(): Boolean = { + epochLock.synchronized { + if (epoch > cacheEpoch) { + cachedSerializedStatuses.clear() + clearCachedBroadcast() + cacheEpoch = epoch + } + cachedSerializedStatuses.get(shuffleId) match { + case Some(bytes) => + retBytes = bytes + true + case None => + logDebug("cached status not found for : " + shuffleId) + statuses = mapStatuses.getOrElse(shuffleId, Array[MapStatus]()) + epochGotten = epoch + false + } } } - // If we got here, we failed to find the serialized locations in the cache, so we pulled - // out a snapshot of the locations as "statuses"; let's serialize and return that - val bytes = MapOutputTracker.serializeMapStatuses(statuses) - logInfo("Size of output statuses for shuffle %d is %d bytes".format(shuffleId, bytes.length)) - // Add them into the table only if the epoch hasn't changed while we were working - epochLock.synchronized { - if (epoch == epochGotten) { - cachedSerializedStatuses(shuffleId) = bytes + + if (checkCachedStatuses()) return retBytes + var shuffleIdLock = shuffleIdLocks.get(shuffleId) + if (null == shuffleIdLock) { + val newLock = new Object() + // in general, this condition should be false - but good to be paranoid + val prevLock = shuffleIdLocks.putIfAbsent(shuffleId, newLock) + shuffleIdLock = if (null != prevLock) prevLock else newLock + } + // synchronize so we only serialize/broadcast it once since multiple threads call + // in parallel + shuffleIdLock.synchronized { + // double check to make sure someone else didn't serialize and cache the same + // mapstatus while we were waiting on the synchronize + if (checkCachedStatuses()) return retBytes + + // If we got here, we failed to find the serialized locations in the cache, so we pulled + // out a snapshot of the locations as "statuses"; let's serialize and return that + val (bytes, bcast) = MapOutputTracker.serializeMapStatuses(statuses, broadcastManager, + isLocal, minSizeForBroadcast) + logInfo("Size of output statuses for shuffle %d is %d bytes".format(shuffleId, bytes.length)) + // Add them into the table only if the epoch hasn't changed while we were working + epochLock.synchronized { + if (epoch == epochGotten) { + cachedSerializedStatuses(shuffleId) = bytes + if (null != bcast) cachedSerializedBroadcast(shuffleId) = bcast + } else { + logInfo("Epoch changed, not caching!") + removeBroadcast(bcast) + } } + bytes } - bytes } override def stop() { + mapOutputRequests.offer(PoisonPill) + threadpool.shutdown() sendTracker(StopMapOutputTracker) mapStatuses.clear() trackerEndpoint = null cachedSerializedStatuses.clear() + clearCachedBroadcast() + shuffleIdLocks.clear() } } @@ -477,12 +598,16 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr private[spark] object MapOutputTracker extends Logging { val ENDPOINT_NAME = "MapOutputTracker" + private val DIRECT = 0 + private val BROADCAST = 1 // Serialize an array of map output locations into an efficient byte format so that we can send // it to reduce tasks. We do this by compressing the serialized bytes using GZIP. They will // generally be pretty compressible because many map outputs will be on the same hostname. - def serializeMapStatuses(statuses: Array[MapStatus]): Array[Byte] = { + def serializeMapStatuses(statuses: Array[MapStatus], broadcastManager: BroadcastManager, + isLocal: Boolean, minBroadcastSize: Int): (Array[Byte], Broadcast[Array[Byte]]) = { val out = new ByteArrayOutputStream + out.write(DIRECT) val objOut = new ObjectOutputStream(new GZIPOutputStream(out)) Utils.tryWithSafeFinally { // Since statuses can be modified in parallel, sync on it @@ -492,16 +617,51 @@ private[spark] object MapOutputTracker extends Logging { } { objOut.close() } - out.toByteArray + val arr = out.toByteArray + if (arr.length >= minBroadcastSize) { + // Use broadcast instead. + // Important arr(0) is the tag == DIRECT, ignore that while deserializing ! + val bcast = broadcastManager.newBroadcast(arr, isLocal) + // toByteArray creates copy, so we can reuse out + out.reset() + out.write(BROADCAST) + val oos = new ObjectOutputStream(new GZIPOutputStream(out)) + oos.writeObject(bcast) + oos.close() + val outArr = out.toByteArray + logInfo("Broadcast mapstatuses size = " + outArr.length + ", actual size = " + arr.length) + (outArr, bcast) + } else { + (arr, null) + } } // Opposite of serializeMapStatuses. def deserializeMapStatuses(bytes: Array[Byte]): Array[MapStatus] = { - val objIn = new ObjectInputStream(new GZIPInputStream(new ByteArrayInputStream(bytes))) - Utils.tryWithSafeFinally { - objIn.readObject().asInstanceOf[Array[MapStatus]] - } { - objIn.close() + assert (bytes.length > 0) + + def deserializeObject(arr: Array[Byte], off: Int, len: Int): AnyRef = { + val objIn = new ObjectInputStream(new GZIPInputStream( + new ByteArrayInputStream(arr, off, len))) + Utils.tryWithSafeFinally { + objIn.readObject() + } { + objIn.close() + } + } + + bytes(0) match { + case DIRECT => + deserializeObject(bytes, 1, bytes.length - 1).asInstanceOf[Array[MapStatus]] + case BROADCAST => + // deserialize the Broadcast, pull .value array out of it, and then deserialize that + val bcast = deserializeObject(bytes, 1, bytes.length - 1). + asInstanceOf[Broadcast[Array[Byte]]] + logInfo("Broadcast mapstatuses size = " + bytes.length + + ", actual size = " + bcast.value.length) + // Important - ignore the DIRECT tag ! Start from offset 1 + deserializeObject(bcast.value, 1, bcast.value.length - 1).asInstanceOf[Array[MapStatus]] + case _ => throw new IllegalArgumentException("Unexpected byte tag = " + bytes(0)) } } diff --git a/core/src/main/scala/org/apache/spark/SSLOptions.scala b/core/src/main/scala/org/apache/spark/SSLOptions.scala index 719905a2c901..be19179b00a4 100644 --- a/core/src/main/scala/org/apache/spark/SSLOptions.scala +++ b/core/src/main/scala/org/apache/spark/SSLOptions.scala @@ -71,7 +71,7 @@ private[spark] case class SSLOptions( keyPassword.foreach(sslContextFactory.setKeyManagerPassword) keyStoreType.foreach(sslContextFactory.setKeyStoreType) if (needClientAuth) { - trustStore.foreach(file => sslContextFactory.setTrustStore(file.getAbsolutePath)) + trustStore.foreach(file => sslContextFactory.setTrustStorePath(file.getAbsolutePath)) trustStorePassword.foreach(sslContextFactory.setTrustStorePassword) trustStoreType.foreach(sslContextFactory.setTrustStoreType) } diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index e8f68224d597..f72c7ded5ea5 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -50,17 +50,19 @@ import org.apache.spark.util.Utils * secure the UI if it has data that other users should not be allowed to see. The javax * servlet filter specified by the user can authenticate the user and then once the user * is logged in, Spark can compare that user versus the view acls to make sure they are - * authorized to view the UI. The configs 'spark.acls.enable' and 'spark.ui.view.acls' - * control the behavior of the acls. Note that the person who started the application - * always has view access to the UI. + * authorized to view the UI. The configs 'spark.acls.enable', 'spark.ui.view.acls' and + * 'spark.ui.view.acls.groups' control the behavior of the acls. Note that the person who + * started the application always has view access to the UI. * - * Spark has a set of modify acls (`spark.modify.acls`) that controls which users have permission - * to modify a single application. This would include things like killing the application. By - * default the person who started the application has modify access. For modify access through - * the UI, you must have a filter that does authentication in place for the modify acls to work - * properly. + * Spark has a set of individual and group modify acls (`spark.modify.acls`) and + * (`spark.modify.acls.groups`) that controls which users and groups have permission to + * modify a single application. This would include things like killing the application. + * By default the person who started the application has modify access. For modify access + * through the UI, you must have a filter that does authentication in place for the modify + * acls to work properly. * - * Spark also has a set of admin acls (`spark.admin.acls`) which is a set of users/administrators + * Spark also has a set of individual and group admin acls (`spark.admin.acls`) and + * (`spark.admin.acls.groups`) which is a set of users/administrators and admin groups * who always have permission to view or modify the Spark application. * * Starting from version 1.3, Spark has partial support for encrypted connections with SSL. @@ -184,6 +186,9 @@ private[spark] class SecurityManager(sparkConf: SparkConf) import SecurityManager._ + // allow all users/groups to have view/modify permissions + private val WILDCARD_ACL = "*" + private val authOn = sparkConf.getBoolean(SecurityManager.SPARK_AUTH_CONF, false) // keep spark.ui.acls.enable for backwards compatibility with 1.0 private var aclsOn = @@ -193,12 +198,20 @@ private[spark] class SecurityManager(sparkConf: SparkConf) private var adminAcls: Set[String] = stringToSet(sparkConf.get("spark.admin.acls", "")) + // admin group acls should be set before view or modify group acls + private var adminAclsGroups : Set[String] = + stringToSet(sparkConf.get("spark.admin.acls.groups", "")) + private var viewAcls: Set[String] = _ + private var viewAclsGroups: Set[String] = _ + // list of users who have permission to modify the application. This should // apply to both UI and CLI for things like killing the application. private var modifyAcls: Set[String] = _ + private var modifyAclsGroups: Set[String] = _ + // always add the current user and SPARK_USER to the viewAcls private val defaultAclUsers = Set[String](System.getProperty("user.name", ""), Utils.getCurrentUserName()) @@ -206,11 +219,16 @@ private[spark] class SecurityManager(sparkConf: SparkConf) setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) setModifyAcls(defaultAclUsers, sparkConf.get("spark.modify.acls", "")) + setViewAclsGroups(sparkConf.get("spark.ui.view.acls.groups", "")); + setModifyAclsGroups(sparkConf.get("spark.modify.acls.groups", "")); + private val secretKey = generateSecretKey() logInfo("SecurityManager: authentication " + (if (authOn) "enabled" else "disabled") + "; ui acls " + (if (aclsOn) "enabled" else "disabled") + - "; users with view permissions: " + viewAcls.toString() + - "; users with modify permissions: " + modifyAcls.toString()) + "; users with view permissions: " + viewAcls.toString() + + "; groups with view permissions: " + viewAclsGroups.toString() + + "; users with modify permissions: " + modifyAcls.toString() + + "; groups with modify permissions: " + modifyAclsGroups.toString()) // Set our own authenticator to properly negotiate user/password for HTTP connections. // This is needed by the HTTP client fetching from the HttpServer. Put here so its @@ -302,17 +320,34 @@ private[spark] class SecurityManager(sparkConf: SparkConf) setViewAcls(Set[String](defaultUser), allowedUsers) } + /** + * Admin acls groups should be set before the view or modify acls groups. If you modify the admin + * acls groups you should also set the view and modify acls groups again to pick up the changes. + */ + def setViewAclsGroups(allowedUserGroups: String) { + viewAclsGroups = (adminAclsGroups ++ stringToSet(allowedUserGroups)); + logInfo("Changing view acls groups to: " + viewAclsGroups.mkString(",")) + } + /** * Checking the existence of "*" is necessary as YARN can't recognize the "*" in "defaultuser,*" */ def getViewAcls: String = { - if (viewAcls.contains("*")) { - "*" + if (viewAcls.contains(WILDCARD_ACL)) { + WILDCARD_ACL } else { viewAcls.mkString(",") } } + def getViewAclsGroups: String = { + if (viewAclsGroups.contains(WILDCARD_ACL)) { + WILDCARD_ACL + } else { + viewAclsGroups.mkString(",") + } + } + /** * Admin acls should be set before the view or modify acls. If you modify the admin * acls you should also set the view and modify acls again to pick up the changes. @@ -322,17 +357,34 @@ private[spark] class SecurityManager(sparkConf: SparkConf) logInfo("Changing modify acls to: " + modifyAcls.mkString(",")) } + /** + * Admin acls groups should be set before the view or modify acls groups. If you modify the admin + * acls groups you should also set the view and modify acls groups again to pick up the changes. + */ + def setModifyAclsGroups(allowedUserGroups: String) { + modifyAclsGroups = (adminAclsGroups ++ stringToSet(allowedUserGroups)); + logInfo("Changing modify acls groups to: " + modifyAclsGroups.mkString(",")) + } + /** * Checking the existence of "*" is necessary as YARN can't recognize the "*" in "defaultuser,*" */ def getModifyAcls: String = { - if (modifyAcls.contains("*")) { - "*" + if (modifyAcls.contains(WILDCARD_ACL)) { + WILDCARD_ACL } else { modifyAcls.mkString(",") } } + def getModifyAclsGroups: String = { + if (modifyAclsGroups.contains(WILDCARD_ACL)) { + WILDCARD_ACL + } else { + modifyAclsGroups.mkString(",") + } + } + /** * Admin acls should be set before the view or modify acls. If you modify the admin * acls you should also set the view and modify acls again to pick up the changes. @@ -342,6 +394,15 @@ private[spark] class SecurityManager(sparkConf: SparkConf) logInfo("Changing admin acls to: " + adminAcls.mkString(",")) } + /** + * Admin acls groups should be set before the view or modify acls groups. If you modify the admin + * acls groups you should also set the view and modify acls groups again to pick up the changes. + */ + def setAdminAclsGroups(adminUserGroups: String) { + adminAclsGroups = stringToSet(adminUserGroups) + logInfo("Changing admin acls groups to: " + adminAclsGroups.mkString(",")) + } + def setAcls(aclSetting: Boolean) { aclsOn = aclSetting logInfo("Changing acls enabled to: " + aclsOn) @@ -398,36 +459,49 @@ private[spark] class SecurityManager(sparkConf: SparkConf) def aclsEnabled(): Boolean = aclsOn /** - * Checks the given user against the view acl list to see if they have + * Checks the given user against the view acl and groups list to see if they have * authorization to view the UI. If the UI acls are disabled * via spark.acls.enable, all users have view access. If the user is null - * it is assumed authentication is off and all users have access. + * it is assumed authentication is off and all users have access. Also if any one of the + * UI acls or groups specify the WILDCARD(*) then all users have view access. * * @param user to see if is authorized * @return true is the user has permission, otherwise false */ def checkUIViewPermissions(user: String): Boolean = { logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " viewAcls=" + - viewAcls.mkString(",")) - !aclsEnabled || user == null || viewAcls.contains(user) || viewAcls.contains("*") + viewAcls.mkString(",") + " viewAclsGroups=" + viewAclsGroups.mkString(",")) + if (!aclsEnabled || user == null || viewAcls.contains(user) || + viewAcls.contains(WILDCARD_ACL) || viewAclsGroups.contains(WILDCARD_ACL)) { + return true + } + val currentUserGroups = Utils.getCurrentUserGroups(sparkConf, user) + logDebug("userGroups=" + currentUserGroups.mkString(",")) + viewAclsGroups.exists(currentUserGroups.contains(_)) } /** - * Checks the given user against the modify acl list to see if they have - * authorization to modify the application. If the UI acls are disabled + * Checks the given user against the modify acl and groups list to see if they have + * authorization to modify the application. If the modify acls are disabled * via spark.acls.enable, all users have modify access. If the user is null - * it is assumed authentication isn't turned on and all users have access. + * it is assumed authentication isn't turned on and all users have access. Also if any one + * of the modify acls or groups specify the WILDCARD(*) then all users have modify access. * * @param user to see if is authorized * @return true is the user has permission, otherwise false */ def checkModifyPermissions(user: String): Boolean = { logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " modifyAcls=" + - modifyAcls.mkString(",")) - !aclsEnabled || user == null || modifyAcls.contains(user) || modifyAcls.contains("*") + modifyAcls.mkString(",") + " modifyAclsGroups=" + modifyAclsGroups.mkString(",")) + if (!aclsEnabled || user == null || modifyAcls.contains(user) || + modifyAcls.contains(WILDCARD_ACL) || modifyAclsGroups.contains(WILDCARD_ACL)) { + return true + } + val currentUserGroups = Utils.getCurrentUserGroups(sparkConf, user) + logDebug("userGroups=" + currentUserGroups) + modifyAclsGroups.exists(currentUserGroups.contains(_)) } - /** * Check to see if authentication for the Spark communication protocols is enabled * @return true if authentication is enabled, otherwise false diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 2cb3ed0296a4..e6cdd0d298f3 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -94,7 +94,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private[spark] val stopped: AtomicBoolean = new AtomicBoolean(false) - private def assertNotStopped(): Unit = { + private[spark] def assertNotStopped(): Unit = { if (stopped.get()) { val activeContext = SparkContext.activeContext.get() val activeCreationSite = @@ -608,6 +608,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * scheduler pool. User-defined properties may also be set here. These properties are propagated * through to worker tasks and can be accessed there via * [[org.apache.spark.TaskContext#getLocalProperty]]. + * + * These properties are inherited by child threads spawned from this thread. This + * may have unexpected consequences when working with thread pools. The standard java + * implementation of thread pools have worker threads spawn other worker threads. + * As a result, local properties may propagate unpredictably. */ def setLocalProperty(key: String, value: String) { if (value == null) { @@ -1219,6 +1224,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" * values to using the `+=` method. Only the driver can access the accumulator's `value`. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]): Accumulator[T] = { val acc = new Accumulator(initialValue, param) cleaner.foreach(_.registerAccumulatorForCleanup(acc.newAcc)) @@ -1230,6 +1236,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * in the Spark UI. Tasks can "add" values to the accumulator using the `+=` method. Only the * driver can access the accumulator's `value`. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]) : Accumulator[T] = { val acc = new Accumulator(initialValue, param, Some(name)) @@ -1243,6 +1250,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @tparam R accumulator result type * @tparam T type that can be added to the accumulator */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]) : Accumulable[R, T] = { val acc = new Accumulable(initialValue, param) @@ -1257,6 +1265,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @tparam R accumulator result type * @tparam T type that can be added to the accumulator */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]) : Accumulable[R, T] = { val acc = new Accumulable(initialValue, param, Some(name)) @@ -1270,6 +1279,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Growable and TraversableOnce are the standard APIs that guarantee += and ++=, implemented by * standard mutable collections. So you can use this with mutable Map, Set, etc. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] (initialValue: R): Accumulable[R, T] = { val param = new GrowableAccumulableParam[R, T] @@ -1282,7 +1292,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Register the given accumulator. Note that accumulators must be registered before use, or it * will throw exception. */ - def register(acc: NewAccumulator[_, _]): Unit = { + def register(acc: AccumulatorV2[_, _]): Unit = { acc.register(this) } @@ -1290,7 +1300,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Register the given accumulator with given name. Note that accumulators must be registered * before use, or it will throw exception. */ - def register(acc: NewAccumulator[_, _], name: String): Unit = { + def register(acc: AccumulatorV2[_, _], name: String): Unit = { acc.register(this, name = Some(name)) } @@ -1330,28 +1340,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli acc } - /** - * Create and register an average accumulator, which accumulates double inputs by recording the - * total sum and total count, and produce the output by sum / total. Note that Double.NaN will be - * returned if no input is added. - */ - def averageAccumulator: AverageAccumulator = { - val acc = new AverageAccumulator - register(acc) - acc - } - - /** - * Create and register an average accumulator, which accumulates double inputs by recording the - * total sum and total count, and produce the output by sum / total. Note that Double.NaN will be - * returned if no input is added. - */ - def averageAccumulator(name: String): AverageAccumulator = { - val acc = new AverageAccumulator - register(acc, name) - acc - } - /** * Create and register a list accumulator, which starts with empty list and accumulates inputs * by adding them into the inner list. diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 27497e21b829..af50a6dc2d8d 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -31,7 +31,6 @@ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.internal.Logging import org.apache.spark.memory.{MemoryManager, StaticMemoryManager, UnifiedMemoryManager} import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.network.BlockTransferService import org.apache.spark.network.netty.NettyBlockTransferService import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.{LiveListenerBus, OutputCommitCoordinator} @@ -61,10 +60,8 @@ class SparkEnv ( val mapOutputTracker: MapOutputTracker, val shuffleManager: ShuffleManager, val broadcastManager: BroadcastManager, - val blockTransferService: BlockTransferService, val blockManager: BlockManager, val securityManager: SecurityManager, - val sparkFilesDir: String, val metricsSystem: MetricsSystem, val memoryManager: MemoryManager, val outputCommitCoordinator: OutputCommitCoordinator, @@ -77,7 +74,7 @@ class SparkEnv ( // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats). private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]() - private var driverTmpDirToDelete: Option[String] = None + private[spark] var driverTmpDir: Option[String] = None private[spark] def stop() { @@ -94,13 +91,10 @@ class SparkEnv ( rpcEnv.shutdown() rpcEnv.awaitTermination() - // Note that blockTransferService is stopped by BlockManager since it is started by it. - // If we only stop sc, but the driver process still run as a services then we need to delete // the tmp dir, if not, it will create too many tmp dirs. - // We only need to delete the tmp dir create by driver, because sparkFilesDir is point to the - // current working dir in executor which we do not need to delete. - driverTmpDirToDelete match { + // We only need to delete the tmp dir create by driver + driverTmpDir match { case Some(path) => try { Utils.deleteRecursively(new File(path)) @@ -284,8 +278,10 @@ object SparkEnv extends Logging { } } + val broadcastManager = new BroadcastManager(isDriver, conf, securityManager) + val mapOutputTracker = if (isDriver) { - new MapOutputTrackerMaster(conf) + new MapOutputTrackerMaster(conf, broadcastManager, isLocal) } else { new MapOutputTrackerWorker(conf) } @@ -325,8 +321,6 @@ object SparkEnv extends Logging { serializerManager, conf, memoryManager, mapOutputTracker, shuffleManager, blockTransferService, securityManager, numUsableCores) - val broadcastManager = new BroadcastManager(isDriver, conf, securityManager) - val metricsSystem = if (isDriver) { // Don't start metrics system right now for Driver. // We need to wait for the task scheduler to give us an app ID. @@ -342,15 +336,6 @@ object SparkEnv extends Logging { ms } - // Set the sparkFiles directory, used when downloading dependencies. In local mode, - // this is a temporary directory; in distributed mode, this is the executor's current working - // directory. - val sparkFilesDir: String = if (isDriver) { - Utils.createTempDir(Utils.getLocalDir(conf), "userFiles").getAbsolutePath - } else { - "." - } - val outputCommitCoordinator = mockOutputCommitCoordinator.getOrElse { new OutputCommitCoordinator(conf, isDriver) } @@ -367,10 +352,8 @@ object SparkEnv extends Logging { mapOutputTracker, shuffleManager, broadcastManager, - blockTransferService, blockManager, securityManager, - sparkFilesDir, metricsSystem, memoryManager, outputCommitCoordinator, @@ -380,7 +363,8 @@ object SparkEnv extends Logging { // called, and we only need to do it for driver. Because driver may run as a service, and if we // don't delete this tmp dir when sc is stopped, then will create too many tmp dirs. if (isDriver) { - envInstance.driverTmpDirToDelete = Some(sparkFilesDir) + val sparkFilesDir = Utils.createTempDir(Utils.getLocalDir(conf), "userFiles").getAbsolutePath + envInstance.driverTmpDir = Some(sparkFilesDir) } envInstance diff --git a/core/src/main/scala/org/apache/spark/SparkFiles.scala b/core/src/main/scala/org/apache/spark/SparkFiles.scala index e85b89fd014e..44f4444a1fa8 100644 --- a/core/src/main/scala/org/apache/spark/SparkFiles.scala +++ b/core/src/main/scala/org/apache/spark/SparkFiles.scala @@ -34,6 +34,6 @@ object SparkFiles { * Get the root directory that contains files added through `SparkContext.addFile()`. */ def getRootDirectory(): String = - SparkEnv.get.sparkFilesDir + SparkEnv.get.driverTmpDir.getOrElse(".") } diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 9e5325746286..27abccf5ac2a 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -24,7 +24,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.source.Source -import org.apache.spark.util.{TaskCompletionListener, TaskFailureListener} +import org.apache.spark.util.{AccumulatorV2, TaskCompletionListener, TaskFailureListener} object TaskContext { @@ -188,6 +188,6 @@ abstract class TaskContext extends Serializable { * Register an accumulator that belongs to this task. Accumulators must call this method when * deserializing in executors. */ - private[spark] def registerAccumulator(a: NewAccumulator[_, _]): Unit + private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit } diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index bc3807f5db18..c904e083911c 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -122,7 +122,7 @@ private[spark] class TaskContextImpl( override def getMetricsSources(sourceName: String): Seq[Source] = metricsSystem.getSourcesByName(sourceName) - private[spark] override def registerAccumulator(a: NewAccumulator[_, _]): Unit = { + private[spark] override def registerAccumulator(a: AccumulatorV2[_, _]): Unit = { taskMetrics.registerAccumulator(a) } diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 82ba2d0c274b..42690844f961 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -23,7 +23,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.Utils +import org.apache.spark.util.{AccumulatorV2, Utils} // ============================================================================================== // NOTE: new task end reasons MUST be accompanied with serialization logic in util.JsonProtocol! @@ -118,7 +118,7 @@ case class ExceptionFailure( fullStackTrace: String, private val exceptionWrapper: Option[ThrowableSerializationWrapper], accumUpdates: Seq[AccumulableInfo] = Seq.empty, - private[spark] var accums: Seq[NewAccumulator[_, _]] = Nil) + private[spark] var accums: Seq[AccumulatorV2[_, _]] = Nil) extends TaskFailedReason { /** @@ -138,7 +138,7 @@ case class ExceptionFailure( this(e, accumUpdates, preserveCause = true) } - private[spark] def withAccums(accums: Seq[NewAccumulator[_, _]]): ExceptionFailure = { + private[spark] def withAccums(accums: Seq[AccumulatorV2[_, _]]): ExceptionFailure = { this.accums = accums this } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 6f3b8faf03b0..e4ccd9f11bd1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -19,7 +19,7 @@ package org.apache.spark.api.java import java.{lang => jl} import java.lang.{Iterable => JIterable} -import java.util.{Comparator, Iterator => JIterator, List => JList} +import java.util.{Comparator, Iterator => JIterator, List => JList, Map => JMap} import scala.collection.JavaConverters._ import scala.reflect.ClassTag @@ -80,7 +80,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * This should ''not'' be called by users directly, but is available for implementors of custom * subclasses of RDD. */ - def iterator(split: Partition, taskContext: TaskContext): java.util.Iterator[T] = + def iterator(split: Partition, taskContext: TaskContext): JIterator[T] = rdd.iterator(split, taskContext).asJava // Transformations (return a new RDD) @@ -96,7 +96,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * of the original partition. */ def mapPartitionsWithIndex[R]( - f: JFunction2[jl.Integer, java.util.Iterator[T], java.util.Iterator[R]], + f: JFunction2[jl.Integer, JIterator[T], JIterator[R]], preservesPartitioning: Boolean = false): JavaRDD[R] = new JavaRDD(rdd.mapPartitionsWithIndex((a, b) => f.call(a, b.asJava).asScala, preservesPartitioning)(fakeClassTag))(fakeClassTag) @@ -147,7 +147,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U]): JavaRDD[U] = { + def mapPartitions[U](f: FlatMapFunction[JIterator[T], U]): JavaRDD[U] = { def fn: (Iterator[T]) => Iterator[U] = { (x: Iterator[T]) => f.call(x.asJava).asScala } @@ -157,7 +157,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U], + def mapPartitions[U](f: FlatMapFunction[JIterator[T], U], preservesPartitioning: Boolean): JavaRDD[U] = { def fn: (Iterator[T]) => Iterator[U] = { (x: Iterator[T]) => f.call(x.asJava).asScala @@ -169,7 +169,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitionsToDouble(f: DoubleFlatMapFunction[java.util.Iterator[T]]): JavaDoubleRDD = { + def mapPartitionsToDouble(f: DoubleFlatMapFunction[JIterator[T]]): JavaDoubleRDD = { def fn: (Iterator[T]) => Iterator[jl.Double] = { (x: Iterator[T]) => f.call(x.asJava).asScala } @@ -179,7 +179,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]): + def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[JIterator[T], K2, V2]): JavaPairRDD[K2, V2] = { def fn: (Iterator[T]) => Iterator[(K2, V2)] = { (x: Iterator[T]) => f.call(x.asJava).asScala @@ -190,7 +190,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitionsToDouble(f: DoubleFlatMapFunction[java.util.Iterator[T]], + def mapPartitionsToDouble(f: DoubleFlatMapFunction[JIterator[T]], preservesPartitioning: Boolean): JavaDoubleRDD = { def fn: (Iterator[T]) => Iterator[jl.Double] = { (x: Iterator[T]) => f.call(x.asJava).asScala @@ -202,7 +202,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2], + def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[JIterator[T], K2, V2], preservesPartitioning: Boolean): JavaPairRDD[K2, V2] = { def fn: (Iterator[T]) => Iterator[(K2, V2)] = { (x: Iterator[T]) => f.call(x.asJava).asScala @@ -214,7 +214,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Applies a function f to each partition of this RDD. */ - def foreachPartition(f: VoidFunction[java.util.Iterator[T]]) { + def foreachPartition(f: VoidFunction[JIterator[T]]): Unit = { rdd.foreachPartition(x => f.call(x.asJava)) } @@ -256,19 +256,33 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return an RDD created by piping elements to a forked external process. */ - def pipe(command: String): JavaRDD[String] = rdd.pipe(command) + def pipe(command: String): JavaRDD[String] = { + rdd.pipe(command) + } /** * Return an RDD created by piping elements to a forked external process. */ - def pipe(command: JList[String]): JavaRDD[String] = + def pipe(command: JList[String]): JavaRDD[String] = { rdd.pipe(command.asScala) + } /** * Return an RDD created by piping elements to a forked external process. */ - def pipe(command: JList[String], env: java.util.Map[String, String]): JavaRDD[String] = + def pipe(command: JList[String], env: JMap[String, String]): JavaRDD[String] = { rdd.pipe(command.asScala, env.asScala) + } + + /** + * Return an RDD created by piping elements to a forked external process. + */ + def pipe(command: JList[String], + env: JMap[String, String], + separateWorkingDir: Boolean, + bufferSize: Int): JavaRDD[String] = { + rdd.pipe(command.asScala, env.asScala, null, null, separateWorkingDir, bufferSize) + } /** * Zips this RDD with another one, returning key-value pairs with the first element in each RDD, @@ -288,7 +302,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def zipPartitions[U, V]( other: JavaRDDLike[U, _], - f: FlatMapFunction2[java.util.Iterator[T], java.util.Iterator[U], V]): JavaRDD[V] = { + f: FlatMapFunction2[JIterator[T], JIterator[U], V]): JavaRDD[V] = { def fn: (Iterator[T], Iterator[U]) => Iterator[V] = { (x: Iterator[T], y: Iterator[U]) => f.call(x.asJava, y.asJava).asScala } @@ -431,6 +445,16 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. + * + * The confidence is the probability that the error bounds of the result will + * contain the true value. That is, if countApprox were called repeatedly + * with confidence 0.9, we would expect 90% of the results to contain the + * true count. The confidence must be in the range [0,1] or an exception will + * be thrown. + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = rdd.countApprox(timeout, confidence) @@ -438,6 +462,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. + * + * @param timeout maximum time to wait for the job, in milliseconds */ def countApprox(timeout: Long): PartialResult[BoundedDouble] = rdd.countApprox(timeout) @@ -446,22 +472,35 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return the count of each unique value in this RDD as a map of (value, count) pairs. The final * combine step happens locally on the master, equivalent to running a single reduce task. */ - def countByValue(): java.util.Map[T, jl.Long] = - mapAsSerializableJavaMap(rdd.countByValue()).asInstanceOf[java.util.Map[T, jl.Long]] + def countByValue(): JMap[T, jl.Long] = + mapAsSerializableJavaMap(rdd.countByValue()).asInstanceOf[JMap[T, jl.Long]] /** - * (Experimental) Approximate version of countByValue(). + * Approximate version of countByValue(). + * + * The confidence is the probability that the error bounds of the result will + * contain the true value. That is, if countApprox were called repeatedly + * with confidence 0.9, we would expect 90% of the results to contain the + * true count. The confidence must be in the range [0,1] or an exception will + * be thrown. + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countByValueApprox( timeout: Long, confidence: Double - ): PartialResult[java.util.Map[T, BoundedDouble]] = + ): PartialResult[JMap[T, BoundedDouble]] = rdd.countByValueApprox(timeout, confidence).map(mapAsSerializableJavaMap) /** - * (Experimental) Approximate version of countByValue(). + * Approximate version of countByValue(). + * + * @param timeout maximum time to wait for the job, in milliseconds + * @return a potentially incomplete result, with error bounds */ - def countByValueApprox(timeout: Long): PartialResult[java.util.Map[T, BoundedDouble]] = + def countByValueApprox(timeout: Long): PartialResult[JMap[T, BoundedDouble]] = rdd.countByValueApprox(timeout).map(mapAsSerializableJavaMap) /** @@ -596,9 +635,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Returns the maximum element from this RDD as defined by the specified * Comparator[T]. + * * @param comp the comparator that defines ordering * @return the maximum of the RDD - * */ + */ def max(comp: Comparator[T]): T = { rdd.max()(Ordering.comparatorToOrdering(comp)) } @@ -606,9 +646,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Returns the minimum element from this RDD as defined by the specified * Comparator[T]. + * * @param comp the comparator that defines ordering * @return the minimum of the RDD - * */ + */ def min(comp: Comparator[T]): T = { rdd.min()(Ordering.comparatorToOrdering(comp)) } @@ -684,7 +725,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * The asynchronous version of the `foreachPartition` action, which * applies a function f to each partition of this RDD. */ - def foreachPartitionAsync(f: VoidFunction[java.util.Iterator[T]]): JavaFutureAction[Void] = { + def foreachPartitionAsync(f: VoidFunction[JIterator[T]]): JavaFutureAction[Void] = { new JavaFutureActionWrapper[Unit, Void](rdd.foreachPartitionAsync(x => f.call(x.asJava)), { x => null.asInstanceOf[Void] }) } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index dfd91ae338e8..fb6323413e3e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -712,8 +712,13 @@ class JavaSparkContext(val sc: SparkContext) } /** - * Set a local property that affects jobs submitted from this thread, such as the - * Spark fair scheduler pool. + * Set a local property that affects jobs submitted from this thread, and all child + * threads, such as the Spark fair scheduler pool. + * + * These properties are inherited by child threads spawned from this thread. This + * may have unexpected consequences when working with thread pools. The standard java + * implementation of thread pools have worker threads spawn other worker threads. + * As a result, local properties may propagate unpredictably. */ def setLocalProperty(key: String, value: String): Unit = sc.setLocalProperty(key, value) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 8bcd2903fe76..64cf4981714c 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -32,7 +32,7 @@ private[spark] object PythonUtils { val pythonPath = new ArrayBuffer[String] for (sparkHome <- sys.env.get("SPARK_HOME")) { pythonPath += Seq(sparkHome, "python", "lib", "pyspark.zip").mkString(File.separator) - pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.9.2-src.zip").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.10.1-src.zip").mkString(File.separator) } pythonPath ++= SparkContext.jarOfObject(this) pythonPath.mkString(File.pathSeparator) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 926e1ff7a874..78606e06fbd3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -43,6 +43,7 @@ import org.apache.ivy.plugins.resolver.{ChainResolver, FileSystemResolver, IBibl import org.apache.spark.{SPARK_VERSION, SparkException, SparkUserAppException} import org.apache.spark.api.r.RUtils import org.apache.spark.deploy.rest._ +import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils} @@ -75,10 +76,6 @@ object SparkSubmit { private val CLUSTER = 2 private val ALL_DEPLOY_MODES = CLIENT | CLUSTER - // A special jar name that indicates the class being run is inside of Spark itself, and therefore - // no user jar is needed. - private val SPARK_INTERNAL = "spark-internal" - // Special primary resource names that represent shells rather than application jars. private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" @@ -305,8 +302,9 @@ object SparkSubmit { } // Require all python files to be local, so we can add them to the PYTHONPATH - // In YARN cluster mode, python files are distributed as regular files, which can be non-local - if (args.isPython && !isYarnCluster) { + // In YARN cluster mode, python files are distributed as regular files, which can be non-local. + // In Mesos cluster mode, non-local python files are automatically downloaded by Mesos. + if (args.isPython && !isYarnCluster && !isMesosCluster) { if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) { printErrorAndExit(s"Only local python files are supported: $args.primaryResource") } @@ -574,7 +572,7 @@ object SparkSubmit { childArgs += ("--primary-r-file", mainFile) childArgs += ("--class", "org.apache.spark.deploy.RRunner") } else { - if (args.primaryResource != SPARK_INTERNAL) { + if (args.primaryResource != SparkLauncher.NO_RESOURCE) { childArgs += ("--jar", args.primaryResource) } childArgs += ("--class", args.mainClass) @@ -794,7 +792,7 @@ object SparkSubmit { } private[deploy] def isInternal(res: String): Boolean = { - res == SPARK_INTERNAL + res == SparkLauncher.NO_RESOURCE } /** @@ -957,9 +955,9 @@ private[spark] object SparkSubmitUtils { // Add scala exclusion rule md.addExcludeRule(createExclusion("*:scala-library:*", ivySettings, ivyConfName)) - // We need to specify each component explicitly, otherwise we miss spark-streaming-kafka and + // We need to specify each component explicitly, otherwise we miss spark-streaming-kafka-0-8 and // other spark-streaming utility components. Underscore is there to differentiate between - // spark-streaming_2.1x and spark-streaming-kafka-assembly_2.1x + // spark-streaming_2.1x and spark-streaming-kafka-0-8-assembly_2.1x val components = Seq("catalyst_", "core_", "graphx_", "hive_", "mllib_", "repl_", "sql_", "streaming_", "yarn_", "network-common_", "network-shuffle_", "network-yarn_") diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 78da1b70c54a..206c130c7637 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -173,6 +173,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .orNull name = Option(name).orElse(sparkProperties.get("spark.app.name")).orNull jars = Option(jars).orElse(sparkProperties.get("spark.jars")).orNull + files = Option(files).orElse(sparkProperties.get("spark.files")).orNull ivyRepoPath = sparkProperties.get("spark.jars.ivy").orNull packages = Option(packages).orElse(sparkProperties.get("spark.jars.packages")).orNull packagesExclusions = Option(packagesExclusions) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 07cbcec8e5f0..110d882f0559 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -245,6 +245,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) ui.getSecurityManager.setAdminAcls(appListener.adminAcls.getOrElse("")) ui.getSecurityManager.setViewAcls(attempt.sparkUser, appListener.viewAcls.getOrElse("")) + ui.getSecurityManager.setAdminAclsGroups(appListener.adminAclsGroups.getOrElse("")) + ui.getSecurityManager.setViewAclsGroups(appListener.viewAclsGroups.getOrElse("")) LoadedAppUI(ui, updateProbe(appId, attemptId, attempt.fileSize)) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index 4ffb5283e99a..53564d0e9515 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -41,7 +41,6 @@ private[spark] class ApplicationInfo( @transient var coresGranted: Int = _ @transient var endTime: Long = _ @transient var appSource: ApplicationSource = _ - @transient @volatile var appUIUrlAtHistoryServer: Option[String] = None // A cap on the number of executors this application can have at any given time. // By default, this is infinite. Only after the first allocation request is issued by the @@ -66,7 +65,6 @@ private[spark] class ApplicationInfo( nextExecutorId = 0 removedExecutors = new ArrayBuffer[ExecutorDesc] executorLimit = desc.initialExecutorLimit.getOrElse(Integer.MAX_VALUE) - appUIUrlAtHistoryServer = None } private def newExecutorId(useID: Option[Int] = None): Int = { @@ -136,11 +134,4 @@ private[spark] class ApplicationInfo( System.currentTimeMillis() - startTime } } - - /** - * Returns the original application UI url unless there is its address at history server - * is defined - */ - def curAppUIUrl: String = appUIUrlAtHistoryServer.getOrElse(desc.appUiUrl) - } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index edc9be2a8a8c..faed4f4dc957 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -17,25 +17,17 @@ package org.apache.spark.deploy.master -import java.io.FileNotFoundException -import java.net.URLEncoder import java.text.SimpleDateFormat import java.util.Date -import java.util.concurrent.{ConcurrentHashMap, ScheduledFuture, TimeUnit} +import java.util.concurrent.{ScheduledFuture, TimeUnit} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.concurrent.{ExecutionContext, Future} -import scala.concurrent.duration.Duration -import scala.language.postfixOps import scala.util.Random -import org.apache.hadoop.fs.Path - import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages._ -import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI @@ -43,9 +35,7 @@ import org.apache.spark.deploy.rest.StandaloneRestServer import org.apache.spark.internal.Logging import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rpc._ -import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.serializer.{JavaSerializer, Serializer} -import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ThreadUtils, Utils} private[deploy] class Master( @@ -59,10 +49,6 @@ private[deploy] class Master( private val forwardMessageThread = ThreadUtils.newDaemonSingleThreadScheduledExecutor("master-forward-message-thread") - private val rebuildUIThread = - ThreadUtils.newDaemonSingleThreadExecutor("master-rebuild-ui-thread") - private val rebuildUIContext = ExecutionContext.fromExecutor(rebuildUIThread) - private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs @@ -85,8 +71,6 @@ private[deploy] class Master( private val addressToApp = new HashMap[RpcAddress, ApplicationInfo] private val completedApps = new ArrayBuffer[ApplicationInfo] private var nextAppNumber = 0 - // Using ConcurrentHashMap so that master-rebuild-ui-thread can add a UI after asyncRebuildUI - private val appIdToUI = new ConcurrentHashMap[String, SparkUI] private val drivers = new HashSet[DriverInfo] private val completedDrivers = new ArrayBuffer[DriverInfo] @@ -199,7 +183,6 @@ private[deploy] class Master( checkForWorkerTimeOutTask.cancel(true) } forwardMessageThread.shutdownNow() - rebuildUIThread.shutdownNow() webUi.stop() restServer.foreach(_.stop()) masterMetricsSystem.stop() @@ -391,9 +374,6 @@ private[deploy] class Master( case CheckForWorkerTimeOut => timeOutDeadWorkers() - case AttachCompletedRebuildUI(appId) => - // An asyncRebuildSparkUI has completed, so need to attach to master webUi - Option(appIdToUI.get(appId)).foreach { ui => webUi.attachSparkUI(ui) } } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -844,7 +824,6 @@ private[deploy] class Master( if (completedApps.size >= RETAINED_APPLICATIONS) { val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) completedApps.take(toRemove).foreach { a => - Option(appIdToUI.remove(a.id)).foreach { ui => webUi.detachSparkUI(ui) } applicationMetricsSystem.removeSource(a.appSource) } completedApps.trimStart(toRemove) @@ -852,9 +831,6 @@ private[deploy] class Master( completedApps += app // Remember it in our history waitingApps -= app - // If application events are logged, use them to rebuild the UI - asyncRebuildSparkUI(app) - for (exec <- app.executors.values) { killExecutor(exec) } @@ -953,89 +929,6 @@ private[deploy] class Master( exec.state = ExecutorState.KILLED } - /** - * Rebuild a new SparkUI from the given application's event logs. - * Return the UI if successful, else None - */ - private[master] def rebuildSparkUI(app: ApplicationInfo): Option[SparkUI] = { - val futureUI = asyncRebuildSparkUI(app) - ThreadUtils.awaitResult(futureUI, Duration.Inf) - } - - /** Rebuild a new SparkUI asynchronously to not block RPC event loop */ - private[master] def asyncRebuildSparkUI(app: ApplicationInfo): Future[Option[SparkUI]] = { - val appName = app.desc.name - val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" - val eventLogDir = app.desc.eventLogDir - .getOrElse { - // Event logging is disabled for this application - app.appUIUrlAtHistoryServer = Some(notFoundBasePath) - return Future.successful(None) - } - val futureUI = Future { - val eventLogFilePrefix = EventLoggingListener.getLogPath( - eventLogDir, app.id, appAttemptId = None, compressionCodecName = app.desc.eventLogCodec) - val fs = Utils.getHadoopFileSystem(eventLogDir, hadoopConf) - val inProgressExists = fs.exists(new Path(eventLogFilePrefix + - EventLoggingListener.IN_PROGRESS)) - - val eventLogFile = if (inProgressExists) { - // Event logging is enabled for this application, but the application is still in progress - logWarning(s"Application $appName is still in progress, it may be terminated abnormally.") - eventLogFilePrefix + EventLoggingListener.IN_PROGRESS - } else { - eventLogFilePrefix - } - - val logInput = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) - val replayBus = new ReplayListenerBus() - val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), - appName, HistoryServer.UI_PATH_PREFIX + s"/${app.id}", app.startTime) - try { - replayBus.replay(logInput, eventLogFile, inProgressExists) - } finally { - logInput.close() - } - - Some(ui) - }(rebuildUIContext) - - futureUI.onSuccess { case Some(ui) => - appIdToUI.put(app.id, ui) - // `self` can be null if we are already in the process of shutting down - // This happens frequently in tests where `local-cluster` is used - if (self != null) { - self.send(AttachCompletedRebuildUI(app.id)) - } - // Application UI is successfully rebuilt, so link the Master UI to it - // NOTE - app.appUIUrlAtHistoryServer is volatile - app.appUIUrlAtHistoryServer = Some(ui.basePath) - }(ThreadUtils.sameThread) - - futureUI.onFailure { - case fnf: FileNotFoundException => - // Event logging is enabled for this application, but no event logs are found - val title = s"Application history not found (${app.id})" - var msg = s"No event logs found for application $appName in ${app.desc.eventLogDir.get}." - logWarning(msg) - msg += " Did you specify the correct logging directory?" - msg = URLEncoder.encode(msg, "UTF-8") - app.appUIUrlAtHistoryServer = Some(notFoundBasePath + s"?msg=$msg&title=$title") - - case e: Exception => - // Relay exception message to application UI page - val title = s"Application history load error (${app.id})" - val exception = URLEncoder.encode(Utils.exceptionString(e), "UTF-8") - var msg = s"Exception in replaying log for application $appName!" - logError(msg, e) - msg = URLEncoder.encode(msg, "UTF-8") - app.appUIUrlAtHistoryServer = - Some(notFoundBasePath + s"?msg=$msg&exception=$exception&title=$title") - }(ThreadUtils.sameThread) - - futureUI - } - /** Generate a new app ID given a app's submission date */ private def newApplicationId(submitDate: Date): String = { val appId = "app-%s-%04d".format(createDateFormat.format(submitDate), nextAppNumber) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala index a055d097674c..a952cee36eb4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala @@ -39,6 +39,4 @@ private[master] object MasterMessages { case object BoundPortsRequest case class BoundPortsResponse(rpcEndpointPort: Int, webUIPort: Int, restPort: Option[Int]) - - case class AttachCompletedRebuildUI(appId: String) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 96274958d142..8875fc223250 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -75,7 +75,11 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")
  • Submit Date: {app.submitDate}
  • State: {app.state}
  • -
  • Application Detail UI
  • + { + if (!app.isFinished) { +
  • Application Detail UI
  • + } + } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala deleted file mode 100644 index e021f1eef794..000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.deploy.master.ui - -import java.net.URLDecoder -import javax.servlet.http.HttpServletRequest - -import scala.xml.Node - -import org.apache.spark.ui.{UIUtils, WebUIPage} - -private[ui] class HistoryNotFoundPage(parent: MasterWebUI) - extends WebUIPage("history/not-found") { - - /** - * Render a page that conveys failure in loading application history. - * - * This accepts 3 HTTP parameters: - * msg = message to display to the user - * title = title of the page - * exception = detailed description of the exception in loading application history (if any) - * - * Parameters "msg" and "exception" are assumed to be UTF-8 encoded. - */ - def render(request: HttpServletRequest): Seq[Node] = { - val titleParam = request.getParameter("title") - val msgParam = request.getParameter("msg") - val exceptionParam = request.getParameter("exception") - - // If no parameters are specified, assume the user did not enable event logging - val defaultTitle = "Event logging is not enabled" - val defaultContent = -
    -
    - No event logs were found for this application! To - enable event logging, - set spark.eventLog.enabled to true and - spark.eventLog.dir to the directory to which your - event logs are written. -
    -
    - - val title = Option(titleParam).getOrElse(defaultTitle) - val content = Option(msgParam) - .map { msg => URLDecoder.decode(msg, "UTF-8") } - .map { msg => -
    -
    {msg}
    -
    ++ - Option(exceptionParam) - .map { e => URLDecoder.decode(e, "UTF-8") } - .map { e =>
    {e}
    } - .getOrElse(Seq.empty) - }.getOrElse(defaultContent) - - UIUtils.basicSparkPage(content, title) - } -} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 363f4b84f885..5ed3e39edc48 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -114,8 +114,8 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {Utils.megabytesToString(aliveWorkers.map(_.memory).sum)} Total, {Utils.megabytesToString(aliveWorkers.map(_.memoryUsed).sum)} Used
  • Applications: - {state.activeApps.length} Running, - {state.completedApps.length} Completed
  • + {state.activeApps.length} Running, + {state.completedApps.length} Completed
  • Drivers: {state.activeDrivers.length} Running, {state.completedDrivers.length} Completed
  • @@ -133,7 +133,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
    -

    Running Applications

    +

    Running Applications

    {activeAppsTable}
    @@ -152,7 +152,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
    -

    Completed Applications

    +

    Completed Applications

    {completedAppsTable}
    @@ -206,7 +206,13 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {killLink} - {app.desc.name} + { + if (app.isFinished) { + app.desc.name + } else { + {app.desc.name} + } + } {app.coresGranted} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index ae16ce90c84b..a0727ad83fb6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -19,8 +19,6 @@ package org.apache.spark.deploy.master.ui import org.apache.spark.deploy.master.Master import org.apache.spark.internal.Logging -import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, ApplicationsListResource, - UIRoot} import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ @@ -30,60 +28,26 @@ import org.apache.spark.ui.JettyUtils._ private[master] class MasterWebUI( val master: Master, - requestedPort: Int, - customMasterPage: Option[MasterPage] = None) + requestedPort: Int) extends WebUI(master.securityMgr, master.securityMgr.getSSLOptions("standalone"), - requestedPort, master.conf, name = "MasterUI") with Logging with UIRoot { + requestedPort, master.conf, name = "MasterUI") with Logging { val masterEndpointRef = master.self val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true) - val masterPage = customMasterPage.getOrElse(new MasterPage(this)) - initialize() /** Initialize all components of the server. */ def initialize() { val masterPage = new MasterPage(this) attachPage(new ApplicationPage(this)) - attachPage(new HistoryNotFoundPage(this)) attachPage(masterPage) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) - attachHandler(ApiRootResource.getServletHandler(this)) attachHandler(createRedirectHandler( "/app/kill", "/", masterPage.handleAppKillRequest, httpMethods = Set("POST"))) attachHandler(createRedirectHandler( "/driver/kill", "/", masterPage.handleDriverKillRequest, httpMethods = Set("POST"))) } - - /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ - def attachSparkUI(ui: SparkUI) { - assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") - ui.getHandlers.foreach(attachHandler) - } - - /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ - def detachSparkUI(ui: SparkUI) { - assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") - ui.getHandlers.foreach(detachHandler) - } - - def getApplicationInfoList: Iterator[ApplicationInfo] = { - val state = masterPage.getMasterState - val activeApps = state.activeApps.sortBy(_.startTime).reverse - val completedApps = state.completedApps.sortBy(_.endTime).reverse - activeApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, false) } ++ - completedApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, true) } - } - - def getSparkUI(appId: String): Option[SparkUI] = { - val state = masterPage.getMasterState - val activeApps = state.activeApps.sortBy(_.startTime).reverse - val completedApps = state.completedApps.sortBy(_.endTime).reverse - (activeApps ++ completedApps).find { _.id == appId }.flatMap { - master.rebuildSparkUI - } - } } private[master] object MasterWebUI { diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index 14244ea5714c..7e93bfc45ebc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -17,13 +17,12 @@ package org.apache.spark.deploy.rest -import java.net.InetSocketAddress import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.io.Source import com.fasterxml.jackson.core.JsonProcessingException -import org.eclipse.jetty.server.Server +import org.eclipse.jetty.server.{Server, ServerConnector} import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.eclipse.jetty.util.thread.QueuedThreadPool import org.json4s._ @@ -80,10 +79,15 @@ private[spark] abstract class RestSubmissionServer( * Return a 2-tuple of the started server and the bound port. */ private def doStart(startPort: Int): (Server, Int) = { - val server = new Server(new InetSocketAddress(host, startPort)) val threadPool = new QueuedThreadPool threadPool.setDaemon(true) - server.setThreadPool(threadPool) + val server = new Server(threadPool) + + val connector = new ServerConnector(server) + connector.setHost(host) + connector.setPort(startPort) + server.addConnector(connector) + val mainHandler = new ServletContextHandler mainHandler.setContextPath("/") contextToServlet.foreach { case (prefix, servlet) => @@ -91,7 +95,7 @@ private[spark] abstract class RestSubmissionServer( } server.setHandler(mainHandler) server.start() - val boundPort = server.getConnectors()(0).getLocalPort + val boundPort = connector.getLocalPort (server, boundPort) } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 4d61f7e23248..9a017f29f7d2 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -23,6 +23,7 @@ import java.net.URL import java.nio.ByteBuffer import java.util.Properties import java.util.concurrent.{ConcurrentHashMap, TimeUnit} +import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -194,6 +195,10 @@ private[spark] class Executor( /** Whether this task has been killed. */ @volatile private var killed = false + /** Whether this task has been finished. */ + @GuardedBy("TaskRunner.this") + private var finished = false + /** How much the JVM process has spent in GC when the task starts to run. */ @volatile var startGCTime: Long = _ @@ -207,10 +212,25 @@ private[spark] class Executor( logInfo(s"Executor is trying to kill $taskName (TID $taskId)") killed = true if (task != null) { - task.kill(interruptThread) + synchronized { + if (!finished) { + task.kill(interruptThread) + } + } } } + /** + * Set the finished flag to true and clear the current thread's interrupt status + */ + private def setTaskFinishedAndClearInterruptStatus(): Unit = synchronized { + this.finished = true + // SPARK-14234 - Reset the interrupted status of the thread to avoid the + // ClosedByInterruptException during execBackend.statusUpdate which causes + // Executor to crash + Thread.interrupted() + } + override def run(): Unit = { val taskMemoryManager = new TaskMemoryManager(env.memoryManager, taskId) val deserializeStartTime = System.currentTimeMillis() @@ -261,20 +281,20 @@ private[spark] class Executor( val releasedLocks = env.blockManager.releaseAllLocksForTask(taskId) val freedMemory = taskMemoryManager.cleanUpAllAllocatedMemory() - if (freedMemory > 0) { + if (freedMemory > 0 && !threwException) { val errMsg = s"Managed memory leak detected; size = $freedMemory bytes, TID = $taskId" - if (conf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false) && !threwException) { + if (conf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false)) { throw new SparkException(errMsg) } else { - logError(errMsg) + logWarning(errMsg) } } - if (releasedLocks.nonEmpty) { + if (releasedLocks.nonEmpty && !threwException) { val errMsg = s"${releasedLocks.size} block locks were not released by TID = $taskId:\n" + releasedLocks.mkString("[", ", ", "]") - if (conf.getBoolean("spark.storage.exceptionOnPinLeak", false) && !threwException) { + if (conf.getBoolean("spark.storage.exceptionOnPinLeak", false)) { throw new SparkException(errMsg) } else { logWarning(errMsg) @@ -336,14 +356,17 @@ private[spark] class Executor( } catch { case ffe: FetchFailedException => val reason = ffe.toTaskEndReason + setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) case _: TaskKilledException | _: InterruptedException if task.killed => logInfo(s"Executor killed $taskName (TID $taskId)") + setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) case CausedBy(cDE: CommitDeniedException) => val reason = cDE.toTaskEndReason + setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) case t: Throwable => @@ -353,7 +376,7 @@ private[spark] class Executor( logError(s"Exception in $taskName (TID $taskId)", t) // Collect latest accumulator values to report back to the driver - val accums: Seq[NewAccumulator[_, _]] = + val accums: Seq[AccumulatorV2[_, _]] = if (task != null) { task.metrics.setExecutorRunTime(System.currentTimeMillis() - taskStart) task.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime) @@ -362,7 +385,7 @@ private[spark] class Executor( Seq.empty } - val accUpdates = accums.map(acc => acc.toInfo(Some(acc.localValue), None)) + val accUpdates = accums.map(acc => acc.toInfo(Some(acc.value), None)) val serializedTaskEndReason = { try { @@ -373,6 +396,7 @@ private[spark] class Executor( ser.serialize(new ExceptionFailure(t, accUpdates, false).withAccums(accums)) } } + setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FAILED, serializedTaskEndReason) // Don't forcibly exit unless the exception was inherently fatal, to avoid @@ -478,7 +502,7 @@ private[spark] class Executor( /** Reports heartbeat and metrics for active tasks to the driver. */ private def reportHeartBeat(): Unit = { // list of (task id, accumUpdates) to send back to the driver - val accumUpdates = new ArrayBuffer[(Long, Seq[NewAccumulator[_, _]])]() + val accumUpdates = new ArrayBuffer[(Long, Seq[AccumulatorV2[_, _]])]() val curGCTime = computeTotalGcTime() for (taskRunner <- runningTasks.values().asScala) { diff --git a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala index 6f7160ac0d3a..3d15f3a0396e 100644 --- a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala @@ -17,8 +17,8 @@ package org.apache.spark.executor -import org.apache.spark.LongAccumulator import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator /** diff --git a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala index db3924cb6937..dada9697c1cf 100644 --- a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala @@ -17,8 +17,8 @@ package org.apache.spark.executor -import org.apache.spark.LongAccumulator import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator /** diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala index fa962108c306..f7a991770d40 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -17,8 +17,8 @@ package org.apache.spark.executor -import org.apache.spark.LongAccumulator import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator /** diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index 0e70a4f52284..ada2e1bc0859 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -17,8 +17,8 @@ package org.apache.spark.executor -import org.apache.spark.LongAccumulator import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator /** diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 0b64917219a7..5bb505bf09f1 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -24,6 +24,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.{BlockId, BlockStatus} +import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, AccumulatorV2, LongAccumulator} /** @@ -98,7 +99,7 @@ class TaskMetrics private[spark] () extends Serializable { /** * Storage statuses of any blocks that have been updated as a result of this task. */ - def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = _updatedBlockStatuses.localValue + def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = _updatedBlockStatuses.value // Setters and increment-ers private[spark] def setExecutorDeserializeTime(v: Long): Unit = @@ -201,7 +202,7 @@ class TaskMetrics private[spark] () extends Serializable { output.RECORDS_WRITTEN -> outputMetrics._recordsWritten ) ++ testAccum.map(TEST_ACCUM -> _) - @transient private[spark] lazy val internalAccums: Seq[NewAccumulator[_, _]] = + @transient private[spark] lazy val internalAccums: Seq[AccumulatorV2[_, _]] = nameToAccums.values.toIndexedSeq /* ========================== * @@ -217,13 +218,13 @@ class TaskMetrics private[spark] () extends Serializable { /** * External accumulators registered with this task. */ - @transient private lazy val externalAccums = new ArrayBuffer[NewAccumulator[_, _]] + @transient private[spark] lazy val externalAccums = new ArrayBuffer[AccumulatorV2[_, _]] - private[spark] def registerAccumulator(a: NewAccumulator[_, _]): Unit = { + private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit = { externalAccums += a } - private[spark] def accumulators(): Seq[NewAccumulator[_, _]] = internalAccums ++ externalAccums + private[spark] def accumulators(): Seq[AccumulatorV2[_, _]] = internalAccums ++ externalAccums } @@ -271,15 +272,15 @@ private[spark] object TaskMetrics extends Logging { /** * Construct a [[TaskMetrics]] object from a list of accumulator updates, called on driver only. */ - def fromAccumulators(accums: Seq[NewAccumulator[_, _]]): TaskMetrics = { + def fromAccumulators(accums: Seq[AccumulatorV2[_, _]]): TaskMetrics = { val tm = new TaskMetrics val (internalAccums, externalAccums) = accums.partition(a => a.name.isDefined && tm.nameToAccums.contains(a.name.get)) internalAccums.foreach { acc => - val tmAcc = tm.nameToAccums(acc.name.get).asInstanceOf[NewAccumulator[Any, Any]] + val tmAcc = tm.nameToAccums(acc.name.get).asInstanceOf[AccumulatorV2[Any, Any]] tmAcc.metadata = acc.metadata - tmAcc.merge(acc.asInstanceOf[NewAccumulator[Any, Any]]) + tmAcc.merge(acc.asInstanceOf[AccumulatorV2[Any, Any]]) } tm.externalAccums ++= externalAccums @@ -289,23 +290,31 @@ private[spark] object TaskMetrics extends Logging { private[spark] class BlockStatusesAccumulator - extends NewAccumulator[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]] { - private[this] var _seq = ArrayBuffer.empty[(BlockId, BlockStatus)] + extends AccumulatorV2[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]] { + private var _seq = ArrayBuffer.empty[(BlockId, BlockStatus)] override def isZero(): Boolean = _seq.isEmpty override def copyAndReset(): BlockStatusesAccumulator = new BlockStatusesAccumulator + override def copy(): BlockStatusesAccumulator = { + val newAcc = new BlockStatusesAccumulator + newAcc._seq = _seq.clone() + newAcc + } + + override def reset(): Unit = _seq.clear() + override def add(v: (BlockId, BlockStatus)): Unit = _seq += v - override def merge(other: NewAccumulator[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]]) + override def merge(other: AccumulatorV2[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]]) : Unit = other match { - case o: BlockStatusesAccumulator => _seq ++= o.localValue + case o: BlockStatusesAccumulator => _seq ++= o.value case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } - override def localValue: Seq[(BlockId, BlockStatus)] = _seq + override def value: Seq[(BlockId, BlockStatus)] = _seq def setValue(newValue: Seq[(BlockId, BlockStatus)]): Unit = { _seq.clear() diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala index 0b552cabfc94..4c6b639015a9 100644 --- a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala @@ -116,13 +116,13 @@ private[memory] class StorageMemoryPool( } /** - * Try to shrink the size of this storage memory pool by `spaceToFree` bytes. Return the number - * of bytes removed from the pool's capacity. + * Free space to shrink the size of this storage memory pool by `spaceToFree` bytes. + * Note: this method doesn't actually reduce the pool size but relies on the caller to do so. + * + * @return number of bytes to be removed from the pool's capacity. */ - def shrinkPoolToFreeSpace(spaceToFree: Long): Long = lock.synchronized { - // First, shrink the pool by reclaiming free memory: + def freeSpaceToShrinkPool(spaceToFree: Long): Long = lock.synchronized { val spaceFreedByReleasingUnusedMemory = math.min(spaceToFree, memoryFree) - decrementPoolSize(spaceFreedByReleasingUnusedMemory) val remainingSpaceToFree = spaceToFree - spaceFreedByReleasingUnusedMemory if (remainingSpaceToFree > 0) { // If reclaiming free memory did not adequately shrink the pool, begin evicting blocks: @@ -130,7 +130,6 @@ private[memory] class StorageMemoryPool( memoryStore.evictBlocksToFreeSpace(None, remainingSpaceToFree, memoryMode) // When a block is released, BlockManager.dropFromMemory() calls releaseMemory(), so we do // not need to decrement _memoryUsed here. However, we do need to decrement the pool size. - decrementPoolSize(spaceFreedByEviction) spaceFreedByReleasingUnusedMemory + spaceFreedByEviction } else { spaceFreedByReleasingUnusedMemory diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index 82023b533d66..ae747c1d163e 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -113,9 +113,10 @@ private[spark] class UnifiedMemoryManager private[memory] ( storagePool.poolSize - storageRegionSize) if (memoryReclaimableFromStorage > 0) { // Only reclaim as much space as is necessary and available: - val spaceReclaimed = storagePool.shrinkPoolToFreeSpace( + val spaceToReclaim = storagePool.freeSpaceToShrinkPool( math.min(extraMemoryNeeded, memoryReclaimableFromStorage)) - executionPool.incrementPoolSize(spaceReclaimed) + storagePool.decrementPoolSize(spaceToReclaim) + executionPool.incrementPoolSize(spaceToReclaim) } } } diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index cc5e7ef3ae00..65a3dd2e37e1 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -43,5 +43,5 @@ package org.apache package object spark { // For package docs only - val SPARK_VERSION = "2.0.0-SNAPSHOT" + val SPARK_VERSION = "2.0.0-preview" } diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index e75f1dbf8107..c19ed1529bbf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -169,42 +169,37 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) var noLocality = true // if true if no preferredLocations exists for parent RDD - // gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones) - def currPrefLocs(part: Partition, prev: RDD[_]): Seq[String] = { - prev.context.getPreferredLocs(prev, part.index).map(tl => tl.host) - } - - // this class just keeps iterating and rotating infinitely over the partitions of the RDD - // next() returns the next preferred machine that a partition is replicated on - // the rotator first goes through the first replica copy of each partition, then second, third - // the iterators return type is a tuple: (replicaString, partition) - class LocationIterator(prev: RDD[_]) extends Iterator[(String, Partition)] { - - var it: Iterator[(String, Partition)] = resetIterator() - - override val isEmpty = !it.hasNext - - // initializes/resets to start iterating from the beginning - def resetIterator(): Iterator[(String, Partition)] = { - val iterators = (0 to 2).map { x => - prev.partitions.iterator.flatMap { p => - if (currPrefLocs(p, prev).size > x) Some((currPrefLocs(p, prev)(x), p)) else None + class PartitionLocations(prev: RDD[_]) { + + // contains all the partitions from the previous RDD that don't have preferred locations + val partsWithoutLocs = ArrayBuffer[Partition]() + // contains all the partitions from the previous RDD that have preferred locations + val partsWithLocs = ArrayBuffer[(String, Partition)]() + + getAllPrefLocs(prev) + + // gets all the preffered locations of the previous RDD and splits them into partitions + // with preferred locations and ones without + def getAllPrefLocs(prev: RDD[_]) { + val tmpPartsWithLocs = mutable.LinkedHashMap[Partition, Seq[String]]() + // first get the locations for each partition, only do this once since it can be expensive + prev.partitions.foreach(p => { + val locs = prev.context.getPreferredLocs(prev, p.index).map(tl => tl.host) + if (locs.size > 0) { + tmpPartsWithLocs.put(p, locs) + } else { + partsWithoutLocs += p + } } - } - iterators.reduceLeft((x, y) => x ++ y) - } - - // hasNext() is false iff there are no preferredLocations for any of the partitions of the RDD - override def hasNext: Boolean = { !isEmpty } - - // return the next preferredLocation of some partition of the RDD - override def next(): (String, Partition) = { - if (it.hasNext) { - it.next() - } else { - it = resetIterator() // ran out of preferred locations, reset and rotate to the beginning - it.next() - } + ) + // convert it into an array of host to partition + (0 to 2).map(x => + tmpPartsWithLocs.foreach(parts => { + val p = parts._1 + val locs = parts._2 + if (locs.size > x) partsWithLocs += ((locs(x), p)) + } ) + ) } } @@ -228,33 +223,32 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) } /** - * Initializes targetLen partition groups and assigns a preferredLocation - * This uses coupon collector to estimate how many preferredLocations it must rotate through - * until it has seen most of the preferred locations (2 * n log(n)) + * Initializes targetLen partition groups. If there are preferred locations, each group + * is assigned a preferredLocation. This uses coupon collector to estimate how many + * preferredLocations it must rotate through until it has seen most of the preferred + * locations (2 * n log(n)) * @param targetLen */ - def setupGroups(targetLen: Int, prev: RDD[_]) { - val rotIt = new LocationIterator(prev) - + def setupGroups(targetLen: Int, partitionLocs: PartitionLocations) { // deal with empty case, just create targetLen partition groups with no preferred location - if (!rotIt.hasNext) { + if (partitionLocs.partsWithLocs.isEmpty) { (1 to targetLen).foreach(x => groupArr += new PartitionGroup()) return } noLocality = false - // number of iterations needed to be certain that we've seen most preferred locations val expectedCoupons2 = 2 * (math.log(targetLen)*targetLen + targetLen + 0.5).toInt var numCreated = 0 var tries = 0 // rotate through until either targetLen unique/distinct preferred locations have been created - // OR we've rotated expectedCoupons2, in which case we have likely seen all preferred locations, - // i.e. likely targetLen >> number of preferred locations (more buckets than there are machines) - while (numCreated < targetLen && tries < expectedCoupons2) { + // OR (we have went through either all partitions OR we've rotated expectedCoupons2 - in + // which case we have likely seen all preferred locations) + val numPartsToLookAt = math.min(expectedCoupons2, partitionLocs.partsWithLocs.length) + while (numCreated < targetLen && tries < numPartsToLookAt) { + val (nxt_replica, nxt_part) = partitionLocs.partsWithLocs(tries) tries += 1 - val (nxt_replica, nxt_part) = rotIt.next() if (!groupHash.contains(nxt_replica)) { val pgroup = new PartitionGroup(Some(nxt_replica)) groupArr += pgroup @@ -263,20 +257,18 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) numCreated += 1 } } - - while (numCreated < targetLen) { // if we don't have enough partition groups, create duplicates - var (nxt_replica, nxt_part) = rotIt.next() + tries = 0 + // if we don't have enough partition groups, create duplicates + while (numCreated < targetLen) { + var (nxt_replica, nxt_part) = partitionLocs.partsWithLocs(tries) + tries += 1 val pgroup = new PartitionGroup(Some(nxt_replica)) groupArr += pgroup groupHash.getOrElseUpdate(nxt_replica, ArrayBuffer()) += pgroup - var tries = 0 - while (!addPartToPGroup(nxt_part, pgroup) && tries < targetLen) { // ensure at least one part - nxt_part = rotIt.next()._2 - tries += 1 - } + addPartToPGroup(nxt_part, pgroup) numCreated += 1 + if (tries >= partitionLocs.partsWithLocs.length) tries = 0 } - } /** @@ -289,10 +281,15 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) * imbalance in favor of locality * @return partition group (bin to be put in) */ - def pickBin(p: Partition, prev: RDD[_], balanceSlack: Double): PartitionGroup = { + def pickBin( + p: Partition, + prev: RDD[_], + balanceSlack: Double, + partitionLocs: PartitionLocations): PartitionGroup = { val slack = (balanceSlack * prev.partitions.length).toInt + val preflocs = partitionLocs.partsWithLocs.filter(_._2 == p).map(_._1).toSeq // least loaded pref locs - val pref = currPrefLocs(p, prev).map(getLeastGroupHash(_)).sortWith(compare) + val pref = preflocs.map(getLeastGroupHash(_)).sortWith(compare) // least loaded pref locs val prefPart = if (pref == Nil) None else pref.head val r1 = rnd.nextInt(groupArr.size) @@ -320,7 +317,10 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) } } - def throwBalls(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { + def throwBalls( + maxPartitions: Int, + prev: RDD[_], + balanceSlack: Double, partitionLocs: PartitionLocations) { if (noLocality) { // no preferredLocations in parent RDD, no randomization needed if (maxPartitions > groupArr.size) { // just return prev.partitions for ((p, i) <- prev.partitions.zipWithIndex) { @@ -334,8 +334,39 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) } } } else { + // It is possible to have unionRDD where one rdd has preferred locations and another rdd + // that doesn't. To make sure we end up with the requested number of partitions, + // make sure to put a partition in every group. + + // if we don't have a partition assigned to every group first try to fill them + // with the partitions with preferred locations + val partIter = partitionLocs.partsWithLocs.iterator + groupArr.filter(pg => pg.numPartitions == 0).foreach { pg => + while (partIter.hasNext && pg.numPartitions == 0) { + var (nxt_replica, nxt_part) = partIter.next() + if (!initialHash.contains(nxt_part)) { + pg.partitions += nxt_part + initialHash += nxt_part + } + } + } + + // if we didn't get one partitions per group from partitions with preferred locations + // use partitions without preferred locations + val partNoLocIter = partitionLocs.partsWithoutLocs.iterator + groupArr.filter(pg => pg.numPartitions == 0).foreach { pg => + while (partNoLocIter.hasNext && pg.numPartitions == 0) { + var nxt_part = partNoLocIter.next() + if (!initialHash.contains(nxt_part)) { + pg.partitions += nxt_part + initialHash += nxt_part + } + } + } + + // finally pick bin for the rest for (p <- prev.partitions if (!initialHash.contains(p))) { // throw every partition into group - pickBin(p, prev, balanceSlack).partitions += p + pickBin(p, prev, balanceSlack, partitionLocs).partitions += p } } } @@ -349,8 +380,11 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) * @return array of partition groups */ def coalesce(maxPartitions: Int, prev: RDD[_]): Array[PartitionGroup] = { - setupGroups(math.min(prev.partitions.length, maxPartitions), prev) // setup the groups (bins) - throwBalls(maxPartitions, prev, balanceSlack) // assign partitions (balls) to each group (bins) + val partitionLocs = new PartitionLocations(prev) + // setup the groups (bins) + setupGroups(math.min(prev.partitions.length, maxPartitions), partitionLocs) + // assign partitions (balls) to each group (bins) + throwBalls(maxPartitions, prev, balanceSlack, partitionLocs) getPartitions } } diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 7936d8e1d45a..3b12448d6393 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -375,6 +375,16 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. + * + * The confidence is the probability that the error bounds of the result will + * contain the true value. That is, if countApprox were called repeatedly + * with confidence 0.9, we would expect 90% of the results to contain the + * true count. The confidence must be in the range [0,1] or an exception will + * be thrown. + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[Map[K, BoundedDouble]] = self.withScope { diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index dd8e46ba0f12..45616856fd7a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -17,9 +17,11 @@ package org.apache.spark.rdd +import java.io.BufferedWriter import java.io.File import java.io.FilenameFilter import java.io.IOException +import java.io.OutputStreamWriter import java.io.PrintWriter import java.util.StringTokenizer import java.util.concurrent.atomic.AtomicReference @@ -45,7 +47,8 @@ private[spark] class PipedRDD[T: ClassTag]( envVars: Map[String, String], printPipeContext: (String => Unit) => Unit, printRDDElement: (T, String => Unit) => Unit, - separateWorkingDir: Boolean) + separateWorkingDir: Boolean, + bufferSize: Int) extends RDD[String](prev) { // Similar to Runtime.exec(), if we are given a single string, split it into words @@ -58,7 +61,7 @@ private[spark] class PipedRDD[T: ClassTag]( printRDDElement: (T, String => Unit) => Unit = null, separateWorkingDir: Boolean = false) = this(prev, PipedRDD.tokenize(command), envVars, printPipeContext, printRDDElement, - separateWorkingDir) + separateWorkingDir, 8192) override def getPartitions: Array[Partition] = firstParent[T].partitions @@ -144,7 +147,8 @@ private[spark] class PipedRDD[T: ClassTag]( new Thread(s"stdin writer for $command") { override def run(): Unit = { TaskContext.setTaskContext(context) - val out = new PrintWriter(proc.getOutputStream) + val out = new PrintWriter(new BufferedWriter( + new OutputStreamWriter(proc.getOutputStream), bufferSize)) try { // scalastyle:off println // input the pipe context firstly diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 499a8b9aa1a8..e6db9b3eec8e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -724,6 +724,7 @@ abstract class RDD[T: ClassTag]( * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = * for (e <- record._2) {f(e)} * @param separateWorkingDir Use separate working directories for each task. + * @param bufferSize Buffer size for the stdin writer for the piped process. * @return the result RDD */ def pipe( @@ -731,11 +732,13 @@ abstract class RDD[T: ClassTag]( env: Map[String, String] = Map(), printPipeContext: (String => Unit) => Unit = null, printRDDElement: (T, String => Unit) => Unit = null, - separateWorkingDir: Boolean = false): RDD[String] = withScope { + separateWorkingDir: Boolean = false, + bufferSize: Int = 8192): RDD[String] = withScope { new PipedRDD(this, command, env, if (printPipeContext ne null) sc.clean(printPipeContext) else null, if (printRDDElement ne null) sc.clean(printRDDElement) else null, - separateWorkingDir) + separateWorkingDir, + bufferSize) } /** @@ -1104,10 +1107,21 @@ abstract class RDD[T: ClassTag]( /** * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. + * + * The confidence is the probability that the error bounds of the result will + * contain the true value. That is, if countApprox were called repeatedly + * with confidence 0.9, we would expect 90% of the results to contain the + * true count. The confidence must be in the range [0,1] or an exception will + * be thrown. + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countApprox( timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = withScope { + require(0.0 <= confidence && confidence <= 1.0, s"confidence ($confidence) must be in [0,1]") val countElements: (TaskContext, Iterator[T]) => Long = { (ctx, iter) => var result = 0L while (iter.hasNext) { @@ -1134,10 +1148,15 @@ abstract class RDD[T: ClassTag]( /** * Approximate version of countByValue(). + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countByValueApprox(timeout: Long, confidence: Double = 0.95) (implicit ord: Ordering[T] = null) : PartialResult[Map[T, BoundedDouble]] = withScope { + require(0.0 <= confidence && confidence <= 1.0, s"confidence ($confidence) must be in [0,1]") if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 66cf4369da2e..8171dcc04637 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -20,6 +20,8 @@ package org.apache.spark.rdd import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer +import scala.collection.parallel.ForkJoinTaskSupport +import scala.concurrent.forkjoin.ForkJoinPool import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} @@ -62,8 +64,22 @@ class UnionRDD[T: ClassTag]( var rdds: Seq[RDD[T]]) extends RDD[T](sc, Nil) { // Nil since we implement getDependencies + // visible for testing + private[spark] val isPartitionListingParallel: Boolean = + rdds.length > conf.getInt("spark.rdd.parallelListingThreshold", 10) + + @transient private lazy val partitionEvalTaskSupport = + new ForkJoinTaskSupport(new ForkJoinPool(8)) + override def getPartitions: Array[Partition] = { - val array = new Array[Partition](rdds.map(_.partitions.length).sum) + val parRDDs = if (isPartitionListingParallel) { + val parArray = rdds.par + parArray.tasksupport = partitionEvalTaskSupport + parArray + } else { + rdds + } + val array = new Array[Partition](parRDDs.map(_.partitions.length).seq.sum) var pos = 0 for ((rdd, rddIndex) <- rdds.zipWithIndex; split <- rdd.partitions) { array(pos) = new UnionPartition(pos, rdd, rddIndex, split.index) diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala index 7d7b4c82fa39..89d2fb9b4797 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala @@ -574,7 +574,7 @@ private[netty] class NettyRpcHandler( private def internalReceive(client: TransportClient, message: ByteBuffer): RequestMessage = { val addr = client.getChannel().remoteAddress().asInstanceOf[InetSocketAddress] assert(addr != null) - val clientAddr = RpcAddress(addr.getHostName, addr.getPort) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) val requestMessage = nettyEnv.deserialize[RequestMessage](client, message) if (requestMessage.senderAddress == null) { // Create a new message with the socket address of the client as the sender. @@ -595,7 +595,7 @@ private[netty] class NettyRpcHandler( override def exceptionCaught(cause: Throwable, client: TransportClient): Unit = { val addr = client.getChannel.remoteAddress().asInstanceOf[InetSocketAddress] if (addr != null) { - val clientAddr = RpcAddress(addr.getHostName, addr.getPort) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) dispatcher.postToAll(RemoteProcessConnectionError(cause, clientAddr)) // If the remove RpcEnv listens to some address, we should also fire a // RemoteProcessConnectionError for the remote RpcEnv listening address @@ -614,14 +614,14 @@ private[netty] class NettyRpcHandler( override def channelActive(client: TransportClient): Unit = { val addr = client.getChannel().remoteAddress().asInstanceOf[InetSocketAddress] assert(addr != null) - val clientAddr = RpcAddress(addr.getHostName, addr.getPort) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) dispatcher.postToAll(RemoteProcessConnected(clientAddr)) } override def channelInactive(client: TransportClient): Unit = { val addr = client.getChannel.remoteAddress().asInstanceOf[InetSocketAddress] if (addr != null) { - val clientAddr = RpcAddress(addr.getHostName, addr.getPort) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) nettyEnv.removeOutbox(clientAddr) dispatcher.postToAll(RemoteProcessDisconnected(clientAddr)) val remoteEnvAddress = remoteAddresses.remove(clientAddr) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index 9f218c64cac2..28c45d800ed0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -32,6 +32,8 @@ private[spark] class ApplicationEventListener extends SparkListener { var endTime: Option[Long] = None var viewAcls: Option[String] = None var adminAcls: Option[String] = None + var viewAclsGroups: Option[String] = None + var adminAclsGroups: Option[String] = None override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { appName = Some(applicationStart.appName) @@ -51,6 +53,8 @@ private[spark] class ApplicationEventListener extends SparkListener { val allProperties = environmentDetails("Spark Properties").toMap viewAcls = allProperties.get("spark.ui.view.acls") adminAcls = allProperties.get("spark.admin.acls") + viewAclsGroups = allProperties.get("spark.ui.view.acls.groups") + adminAclsGroups = allProperties.get("spark.admin.acls.groups") } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index a96d5f6fbf08..5291b663667e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -209,7 +209,7 @@ class DAGScheduler( task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Seq[NewAccumulator[_, _]], + accumUpdates: Seq[AccumulatorV2[_, _]], taskInfo: TaskInfo): Unit = { eventProcessLoop.post( CompletionEvent(task, reason, result, accumUpdates, taskInfo)) @@ -286,7 +286,9 @@ class DAGScheduler( case None => // We are going to register ancestor shuffle dependencies getAncestorShuffleDependencies(shuffleDep.rdd).foreach { dep => - shuffleToMapStage(dep.shuffleId) = newOrUsedShuffleStage(dep, firstJobId) + if (!shuffleToMapStage.contains(dep.shuffleId)) { + shuffleToMapStage(dep.shuffleId) = newOrUsedShuffleStage(dep, firstJobId) + } } // Then register current shuffleDep val stage = newOrUsedShuffleStage(shuffleDep, firstJobId) @@ -1091,14 +1093,14 @@ class DAGScheduler( event.accumUpdates.foreach { updates => val id = updates.id // Find the corresponding accumulator on the driver and update it - val acc: NewAccumulator[Any, Any] = AccumulatorContext.get(id) match { - case Some(accum) => accum.asInstanceOf[NewAccumulator[Any, Any]] + val acc: AccumulatorV2[Any, Any] = AccumulatorContext.get(id) match { + case Some(accum) => accum.asInstanceOf[AccumulatorV2[Any, Any]] case None => throw new SparkException(s"attempted to access non-existent accumulator $id") } - acc.merge(updates.asInstanceOf[NewAccumulator[Any, Any]]) + acc.merge(updates.asInstanceOf[AccumulatorV2[Any, Any]]) // To avoid UI cruft, ignore cases where value wasn't updated - if (acc.name.isDefined && !updates.isZero()) { + if (acc.name.isDefined && !updates.isZero) { stage.latestInfo.accumulables(id) = acc.toInfo(None, Some(acc.value)) event.taskInfo.accumulables += acc.toInfo(Some(updates.value), Some(acc.value)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index e57a2246d872..8c761124824a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -23,7 +23,7 @@ import scala.language.existentials import org.apache.spark._ import org.apache.spark.rdd.RDD -import org.apache.spark.util.CallSite +import org.apache.spark.util.{AccumulatorV2, CallSite} /** * Types of events that can be handled by the DAGScheduler. The DAGScheduler uses an event queue @@ -71,7 +71,7 @@ private[scheduler] case class CompletionEvent( task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Seq[NewAccumulator[_, _]], + accumUpdates: Seq[AccumulatorV2[_, _]], taskInfo: TaskInfo) extends DAGSchedulerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index a79e71ec7c9b..5987cfea2e9b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -26,16 +26,14 @@ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode /** - * An Schedulable entity that represent collection of Pools or TaskSetManagers + * An Schedulable entity that represents collection of Pools or TaskSetManagers */ - private[spark] class Pool( val poolName: String, val schedulingMode: SchedulingMode, initMinShare: Int, initWeight: Int) - extends Schedulable - with Logging { + extends Schedulable with Logging { val schedulableQueue = new ConcurrentLinkedQueue[Schedulable] val schedulableNameToSchedulable = new ConcurrentHashMap[String, Schedulable] @@ -56,7 +54,8 @@ private[spark] class Pool( case SchedulingMode.FIFO => new FIFOSchedulingAlgorithm() case _ => - throw new IllegalArgumentException(s"Unsupported spark.scheduler.mode: $schedulingMode") + val msg = "Unsupported scheduling mode: $schedulingMode. Use FAIR or FIFO instead." + throw new IllegalArgumentException(msg) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala index 864941d468af..18ebbbe78a5b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala @@ -36,11 +36,7 @@ private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { val stageId2 = s2.stageId res = math.signum(stageId1 - stageId2) } - if (res < 0) { - true - } else { - false - } + res < 0 } } @@ -52,12 +48,12 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { val runningTasks2 = s2.runningTasks val s1Needy = runningTasks1 < minShare1 val s2Needy = runningTasks2 < minShare2 - val minShareRatio1 = runningTasks1.toDouble / math.max(minShare1, 1.0).toDouble - val minShareRatio2 = runningTasks2.toDouble / math.max(minShare2, 1.0).toDouble + val minShareRatio1 = runningTasks1.toDouble / math.max(minShare1, 1.0) + val minShareRatio2 = runningTasks2.toDouble / math.max(minShare2, 1.0) val taskToWeightRatio1 = runningTasks1.toDouble / s1.weight.toDouble val taskToWeightRatio2 = runningTasks2.toDouble / s2.weight.toDouble - var compare: Int = 0 + var compare = 0 if (s1Needy && !s2Needy) { return true } else if (!s1Needy && s2Needy) { @@ -67,7 +63,6 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { } else { compare = taskToWeightRatio1.compareTo(taskToWeightRatio2) } - if (compare < 0) { true } else if (compare > 0) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index e7ca6efd84ae..15f863b66c6e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -28,7 +28,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.{MemoryMode, TaskMemoryManager} import org.apache.spark.metrics.MetricsSystem import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Utils} +import org.apache.spark.util.{AccumulatorV2, ByteBufferInputStream, ByteBufferOutputStream, Utils} /** * A unit of execution. We have two kinds of Task's in Spark: @@ -153,9 +153,16 @@ private[spark] abstract class Task[T]( * Collect the latest values of accumulators used in this task. If the task failed, * filter out the accumulators whose values should not be included on failures. */ - def collectAccumulatorUpdates(taskFailed: Boolean = false): Seq[NewAccumulator[_, _]] = { + def collectAccumulatorUpdates(taskFailed: Boolean = false): Seq[AccumulatorV2[_, _]] = { if (context != null) { - context.taskMetrics.accumulators().filter { a => !taskFailed || a.countFailedValues } + context.taskMetrics.internalAccums.filter { a => + // RESULT_SIZE accumulator is always zero at executor, we need to send it back as its + // value will be updated at driver side. + // Note: internal accumulators representing task metrics always count failed values + !a.isZero || a.name == Some(InternalAccumulator.RESULT_SIZE) + // zero value external accumulators may still be useful, e.g. SQLMetrics, we should not filter + // them out. + } ++ context.taskMetrics.externalAccums.filter(a => !taskFailed || a.countFailedValues) } else { Seq.empty } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index b472c5511b73..80f2bf41224b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -22,9 +22,9 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{NewAccumulator, SparkEnv} +import org.apache.spark.SparkEnv import org.apache.spark.storage.BlockId -import org.apache.spark.util.Utils +import org.apache.spark.util.{AccumulatorV2, Utils} // Task result. Also contains updates to accumulator variables. private[spark] sealed trait TaskResult[T] @@ -36,7 +36,7 @@ private[spark] case class IndirectTaskResult[T](blockId: BlockId, size: Int) /** A TaskResult that contains the task's return value and accumulator updates. */ private[spark] class DirectTaskResult[T]( var valueBytes: ByteBuffer, - var accumUpdates: Seq[NewAccumulator[_, _]]) + var accumUpdates: Seq[AccumulatorV2[_, _]]) extends TaskResult[T] with Externalizable { private var valueObjectDeserialized = false @@ -61,9 +61,9 @@ private[spark] class DirectTaskResult[T]( if (numUpdates == 0) { accumUpdates = null } else { - val _accumUpdates = new ArrayBuffer[NewAccumulator[_, _]] + val _accumUpdates = new ArrayBuffer[AccumulatorV2[_, _]] for (i <- 0 until numUpdates) { - _accumUpdates += in.readObject.asInstanceOf[NewAccumulator[_, _]] + _accumUpdates += in.readObject.asInstanceOf[AccumulatorV2[_, _]] } accumUpdates = _accumUpdates } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index b438c285fdf1..685ef55c6687 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -27,7 +27,7 @@ import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.internal.Logging import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{LongAccumulator, ThreadUtils, Utils} /** * Runs a thread pool that deserializes and remotely fetches (if necessary) task results. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 75a0c5631197..cd13eebe74a9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -17,9 +17,9 @@ package org.apache.spark.scheduler -import org.apache.spark.NewAccumulator import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.AccumulatorV2 /** * Low-level task scheduler interface, currently implemented exclusively by @@ -67,7 +67,7 @@ private[spark] trait TaskScheduler { */ def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 8fa4aa121c12..371fb8602f78 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -34,7 +34,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils} /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. @@ -384,17 +384,12 @@ private[spark] class TaskSchedulerImpl( */ override def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean = { // (taskId, stageId, stageAttemptId, accumUpdates) val accumUpdatesWithTaskIds: Array[(Long, Int, Int, Seq[AccumulableInfo])] = synchronized { accumUpdates.flatMap { case (id, updates) => - // We should call `acc.value` here as we are at driver side now. However, the RPC framework - // optimizes local message delivery so that messages do not need to de serialized and - // deserialized. This brings trouble to the accumulator framework, which depends on - // serialization to set the `atDriverSide` flag. Here we call `acc.localValue` instead to - // be more robust about this issue. - val accInfos = updates.map(acc => acc.toInfo(Some(acc.localValue), None)) + val accInfos = updates.map(acc => acc.toInfo(Some(acc.value), None)) taskIdToTaskSetManager.get(id).map { taskSetMgr => (id, taskSetMgr.stageId, taskSetMgr.taskSet.stageAttemptId, accInfos) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index b79f643a7481..08d33f688a16 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -32,7 +32,7 @@ import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode._ import org.apache.spark.TaskState.TaskState -import org.apache.spark.util.{Clock, SystemClock, Utils} +import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, Utils} /** * Schedules the tasks within a single TaskSet in the TaskSchedulerImpl. This class keeps track of @@ -479,7 +479,7 @@ private[spark] class TaskSetManager( // val timeTaken = clock.getTime() - startTime val taskName = s"task ${info.id} in stage ${taskSet.id}" logInfo(s"Starting $taskName (TID $taskId, $host, partition ${task.partitionId}," + - s"$taskLocality, ${serializedTask.limit} bytes)") + s" $taskLocality, ${serializedTask.limit} bytes)") sched.dagScheduler.taskStarted(task, info) return Some(new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId, @@ -647,7 +647,7 @@ private[spark] class TaskSetManager( info.markFailed() val index = info.index copiesRunning(index) -= 1 - var accumUpdates: Seq[NewAccumulator[_, _]] = Seq.empty + var accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}): " + reason.asInstanceOf[TaskFailedReason].toErrorString val failureException: Option[Throwable] = reason match { @@ -716,7 +716,16 @@ private[spark] class TaskSetManager( failedExecutors.getOrElseUpdate(index, new HashMap[String, Long]()). put(info.executorId, clock.getTimeMillis()) sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, info) - addPendingTask(index) + + if (successful(index)) { + logInfo( + s"Task ${info.id} in stage ${taskSet.id} (TID $tid) failed, " + + "but another instance of the task has already succeeded, " + + "so not re-queuing the task to be re-executed.") + } else { + addPendingTask(index) + } + if (!isZombie && state != TaskState.KILLED && reason.isInstanceOf[TaskFailedReason] && reason.asInstanceOf[TaskFailedReason].countTowardsTaskFailures) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 8896391f9775..0fea9c123bcf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -289,7 +289,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp scheduler.executorLost(executorId, if (killed) ExecutorKilled else reason) listenerBus.post( SparkListenerExecutorRemoved(System.currentTimeMillis(), executorId, reason.toString)) - case None => logInfo(s"Asked to remove non-existent executor $executorId") + case None => + // SPARK-15262: If an executor is still alive even after the scheduler has removed + // its metadata, we may receive a heartbeat from that executor and tell its block + // manager to reregister itself. If that happens, the block manager master will know + // about the executor, but the scheduler will not. Therefore, we should remove the + // executor from the block manager when we hit this case. + scheduler.sc.env.blockManager.master.removeExecutor(executorId) + logInfo(s"Asked to remove non-existent executor $executorId") } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 50b452c72f8a..2c5be1f52889 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -109,10 +109,14 @@ private[spark] class CoarseMesosSchedulerBackend( private val slaveOfferConstraints = parseConstraintString(sc.conf.get("spark.mesos.constraints", "")) - // reject offers with mismatched constraints in seconds + // Reject offers with mismatched constraints in seconds private val rejectOfferDurationForUnmetConstraints = getRejectOfferDurationForUnmetConstraints(sc) + // Reject offers when we reached the maximum number of cores for this framework + private val rejectOfferDurationForReachedMaxCores = + getRejectOfferDurationForReachedMaxCores(sc) + // A client for talking to the external shuffle service private val mesosExternalShuffleClient: Option[MesosExternalShuffleClient] = { if (shuffleServiceEnabled) { @@ -279,18 +283,32 @@ private[spark] class CoarseMesosSchedulerBackend( } private def declineUnmatchedOffers(d: SchedulerDriver, offers: Buffer[Offer]): Unit = { - for (offer <- offers) { - val id = offer.getId.getValue - val offerAttributes = toAttributeMap(offer.getAttributesList) - val mem = getResource(offer.getResourcesList, "mem") - val cpus = getResource(offer.getResourcesList, "cpus") - val filters = Filters.newBuilder() - .setRefuseSeconds(rejectOfferDurationForUnmetConstraints).build() - - logDebug(s"Declining offer: $id with attributes: $offerAttributes mem: $mem cpu: $cpus" - + s" for $rejectOfferDurationForUnmetConstraints seconds") + offers.foreach { offer => + declineOffer(d, offer, Some("unmet constraints"), + Some(rejectOfferDurationForUnmetConstraints)) + } + } - d.declineOffer(offer.getId, filters) + private def declineOffer( + d: SchedulerDriver, + offer: Offer, + reason: Option[String] = None, + refuseSeconds: Option[Long] = None): Unit = { + + val id = offer.getId.getValue + val offerAttributes = toAttributeMap(offer.getAttributesList) + val mem = getResource(offer.getResourcesList, "mem") + val cpus = getResource(offer.getResourcesList, "cpus") + + logDebug(s"Declining offer: $id with attributes: $offerAttributes mem: $mem" + + s" cpu: $cpus for $refuseSeconds seconds" + + reason.map(r => s" (reason: $r)").getOrElse("")) + + refuseSeconds match { + case Some(seconds) => + val filters = Filters.newBuilder().setRefuseSeconds(seconds).build() + d.declineOffer(offer.getId, filters) + case _ => d.declineOffer(offer.getId) } } @@ -326,11 +344,12 @@ private[spark] class CoarseMesosSchedulerBackend( d.launchTasks( Collections.singleton(offer.getId), offerTasks.asJava) - } else { // decline - logDebug(s"Declining offer: $id with attributes: $offerAttributes " + - s"mem: $offerMem cpu: $offerCpus") - - d.declineOffer(offer.getId) + } else if (totalCoresAcquired >= maxCores) { + // Reject an offer for a configurable amount of time to avoid starving other frameworks + declineOffer(d, offer, Some("reached spark.cores.max"), + Some(rejectOfferDurationForReachedMaxCores)) + } else { + declineOffer(d, offer) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 1e322ac67941..7355ba317d9a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -352,4 +352,8 @@ private[mesos] trait MesosSchedulerUtils extends Logging { sc.conf.getTimeAsSeconds("spark.mesos.rejectOfferDurationForUnmetConstraints", "120s") } + protected def getRejectOfferDurationForReachedMaxCores(sc: SparkContext): Long = { + sc.conf.getTimeAsSeconds("spark.mesos.rejectOfferDurationForReachedMaxCores", "120s") + } + } diff --git a/core/src/main/scala/org/apache/spark/security/GroupMappingServiceProvider.scala b/core/src/main/scala/org/apache/spark/security/GroupMappingServiceProvider.scala new file mode 100644 index 000000000000..ea047a4f75d5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/security/GroupMappingServiceProvider.scala @@ -0,0 +1,38 @@ +/* + * 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.security + +/** + * This Spark trait is used for mapping a given userName to a set of groups which it belongs to. + * This is useful for specifying a common group of admins/developers to provide them admin, modify + * and/or view access rights. Based on whether access control checks are enabled using + * spark.acls.enable, every time a user tries to access or modify the application, the + * SecurityManager gets the corresponding groups a user belongs to from the instance of the groups + * mapping provider specified by the entry spark.user.groups.mapping. + */ + +trait GroupMappingServiceProvider { + + /** + * Get the groups the user belongs to. + * @param userName User's Name + * @return set of groups that the user belongs to. Empty in case of an invalid user. + */ + def getGroups(userName : String) : Set[String] + +} diff --git a/core/src/main/scala/org/apache/spark/security/ShellBasedGroupsMappingProvider.scala b/core/src/main/scala/org/apache/spark/security/ShellBasedGroupsMappingProvider.scala new file mode 100644 index 000000000000..f71dd08246b2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/security/ShellBasedGroupsMappingProvider.scala @@ -0,0 +1,45 @@ +/* + * 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.security + +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +/** + * This class is responsible for getting the groups for a particular user in Unix based + * environments. This implementation uses the Unix Shell based id command to fetch the user groups + * for the specified user. It does not cache the user groups as the invocations are expected + * to be infrequent. + */ + +private[spark] class ShellBasedGroupsMappingProvider extends GroupMappingServiceProvider + with Logging { + + override def getGroups(username: String): Set[String] = { + val userGroups = getUnixGroups(username) + logDebug("User: " + username + " Groups: " + userGroups.mkString(",")) + userGroups + } + + // shells out a "bash -c id -Gn username" to get user groups + private def getUnixGroups(username: String): Set[String] = { + val cmdSeq = Seq("bash", "-c", "id -Gn " + username) + // we need to get rid of the trailing "\n" from the result of command execution + Utils.executeAndGetOutput(cmdSeq).stripLineEnd.split(" ").toSet + } +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala index eddc36edc961..7d63a8f734f0 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala @@ -20,10 +20,10 @@ import java.util.{Arrays, Date, List => JList} import javax.ws.rs.{GET, Produces, QueryParam} import javax.ws.rs.core.MediaType -import org.apache.spark.executor.{InputMetrics => InternalInputMetrics, OutputMetrics => InternalOutputMetrics, ShuffleReadMetrics => InternalShuffleReadMetrics, ShuffleWriteMetrics => InternalShuffleWriteMetrics, TaskMetrics => InternalTaskMetrics} import org.apache.spark.scheduler.{AccumulableInfo => InternalAccumulableInfo, StageInfo} import org.apache.spark.ui.SparkUI import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData} +import org.apache.spark.ui.jobs.UIData.{InputMetricsUIData => InternalInputMetrics, OutputMetricsUIData => InternalOutputMetrics, ShuffleReadMetricsUIData => InternalShuffleReadMetrics, ShuffleWriteMetricsUIData => InternalShuffleWriteMetrics, TaskMetricsUIData => InternalTaskMetrics} import org.apache.spark.util.Distribution @Produces(Array(MediaType.APPLICATION_JSON)) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index ba9cd711f18e..681f295006e3 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -21,10 +21,10 @@ import javax.servlet.ServletContext import javax.ws.rs._ import javax.ws.rs.core.{Context, Response} -import com.sun.jersey.api.core.ResourceConfig -import com.sun.jersey.spi.container.servlet.ServletContainer import org.eclipse.jetty.server.handler.ContextHandler import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} +import org.glassfish.jersey.server.ServerProperties +import org.glassfish.jersey.servlet.ServletContainer import org.apache.spark.SecurityManager import org.apache.spark.ui.SparkUI @@ -191,12 +191,7 @@ private[spark] object ApiRootResource { val jerseyContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS) jerseyContext.setContextPath("/api") val holder: ServletHolder = new ServletHolder(classOf[ServletContainer]) - holder.setInitParameter("com.sun.jersey.config.property.resourceConfigClass", - "com.sun.jersey.api.core.PackagesResourceConfig") - holder.setInitParameter("com.sun.jersey.config.property.packages", - "org.apache.spark.status.api.v1") - holder.setInitParameter(ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS, - classOf[SecurityFilter].getCanonicalName) + holder.setInitParameter(ServerProperties.PROVIDER_PACKAGES, "org.apache.spark.status.api.v1") UIRootFromServletContext.setUiRoot(jerseyContext, uiRoot) jerseyContext.addServlet(holder, "/*") jerseyContext @@ -205,7 +200,7 @@ private[spark] object ApiRootResource { /** * This trait is shared by the all the root containers for application UI information -- - * the HistoryServer, the Master UI, and the application UI. This provides the common + * the HistoryServer and the application UI. This provides the common * interface needed for them all to expose application info as json. */ private[spark] trait UIRoot { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala index 0f3018368246..02fd2985fa20 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala @@ -21,7 +21,6 @@ import javax.ws.rs.{DefaultValue, GET, Produces, QueryParam} import javax.ws.rs.core.MediaType import org.apache.spark.deploy.history.ApplicationHistoryInfo -import org.apache.spark.deploy.master.{ApplicationInfo => InternalApplicationInfo} @Produces(Array(MediaType.APPLICATION_JSON)) private[v1] class ApplicationListResource(uiRoot: UIRoot) { @@ -84,33 +83,4 @@ private[spark] object ApplicationsListResource { } ) } - - def convertApplicationInfo( - internal: InternalApplicationInfo, - completed: Boolean): ApplicationInfo = { - // standalone application info always has just one attempt - new ApplicationInfo( - id = internal.id, - name = internal.desc.name, - coresGranted = Some(internal.coresGranted), - maxCores = internal.desc.maxCores, - coresPerExecutor = internal.desc.coresPerExecutor, - memoryPerExecutorMB = Some(internal.desc.memoryPerExecutorMB), - attempts = Seq(new ApplicationAttemptInfo( - attemptId = None, - startTime = new Date(internal.startTime), - endTime = new Date(internal.endTime), - duration = - if (internal.endTime > 0) { - internal.endTime - internal.startTime - } else { - 0 - }, - lastUpdated = new Date(internal.endTime), - sparkUser = internal.desc.user, - completed = completed - )) - ) - } - } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala index 95fbd96ade5a..b4a991eda35f 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala @@ -16,18 +16,16 @@ */ package org.apache.spark.status.api.v1 -import javax.ws.rs.WebApplicationException +import javax.ws.rs.container.{ContainerRequestContext, ContainerRequestFilter} import javax.ws.rs.core.Response +import javax.ws.rs.ext.Provider -import com.sun.jersey.spi.container.{ContainerRequest, ContainerRequestFilter} - +@Provider private[v1] class SecurityFilter extends ContainerRequestFilter with UIRootFromServletContext { - def filter(req: ContainerRequest): ContainerRequest = { - val user = Option(req.getUserPrincipal).map { _.getName }.orNull - if (uiRoot.securityManager.checkUIViewPermissions(user)) { - req - } else { - throw new WebApplicationException( + override def filter(req: ContainerRequestContext): Unit = { + val user = Option(req.getSecurityContext.getUserPrincipal).map { _.getName }.orNull + if (!uiRoot.securityManager.checkUIViewPermissions(user)) { + req.abortWith( Response .status(Response.Status.FORBIDDEN) .entity(raw"""user "$user"is not authorized""") diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index ca53534b61c4..20ffe1342e50 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -228,10 +228,7 @@ private[storage] class BlockInfoManager extends Logging { infos.get(blockId) match { case None => return None case Some(info) => - if (info.writerTask == currentTaskAttemptId) { - throw new IllegalStateException( - s"Task $currentTaskAttemptId has already locked $blockId for writing") - } else if (info.writerTask == BlockInfo.NO_WRITER && info.readerCount == 0) { + if (info.writerTask == BlockInfo.NO_WRITER && info.readerCount == 0) { info.writerTask = currentTaskAttemptId writeLocksByTask.addBinding(currentTaskAttemptId, blockId) logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId") diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index f2d06c7ea807..c56e451c11cb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -65,7 +65,7 @@ private[spark] class BlockManager( memoryManager: MemoryManager, mapOutputTracker: MapOutputTracker, shuffleManager: ShuffleManager, - blockTransferService: BlockTransferService, + val blockTransferService: BlockTransferService, securityManager: SecurityManager, numUsableCores: Int) extends BlockDataManager with BlockEvictionHandler with Logging { diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index db24f0319ba0..6854f7baf716 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -25,11 +25,10 @@ import scala.collection.mutable.ArrayBuffer import scala.language.implicitConversions import scala.xml.Node -import org.eclipse.jetty.server.{Connector, Request, Server} +import org.eclipse.jetty.server.{Request, Server, ServerConnector} import org.eclipse.jetty.server.handler._ -import org.eclipse.jetty.server.nio.SelectChannelConnector -import org.eclipse.jetty.server.ssl.SslSelectChannelConnector import org.eclipse.jetty.servlet._ +import org.eclipse.jetty.servlets.gzip.GzipHandler import org.eclipse.jetty.util.component.LifeCycle import org.eclipse.jetty.util.thread.QueuedThreadPool import org.json4s.JValue @@ -243,10 +242,16 @@ private[spark] object JettyUtils extends Logging { // Bind to the given port, or throw a java.net.BindException if the port is occupied def connect(currentPort: Int): (Server, Int) = { - val server = new Server - val connectors = new ArrayBuffer[Connector] + val pool = new QueuedThreadPool + if (serverName.nonEmpty) { + pool.setName(serverName) + } + pool.setDaemon(true) + + val server = new Server(pool) + val connectors = new ArrayBuffer[ServerConnector] // Create a connector on port currentPort to listen for HTTP requests - val httpConnector = new SelectChannelConnector() + val httpConnector = new ServerConnector(server) httpConnector.setPort(currentPort) connectors += httpConnector @@ -260,8 +265,9 @@ private[spark] object JettyUtils extends Logging { } val scheme = "https" // Create a connector on port securePort to listen for HTTPS requests - val connector = new SslSelectChannelConnector(factory) + val connector = new ServerConnector(server, factory) connector.setPort(securePort) + connectors += connector // redirect the HTTP requests to HTTPS port @@ -269,34 +275,27 @@ private[spark] object JettyUtils extends Logging { } gzipHandlers.foreach(collection.addHandler) - connectors.foreach(_.setHost(hostName)) // As each acceptor and each selector will use one thread, the number of threads should at // least be the number of acceptors and selectors plus 1. (See SPARK-13776) var minThreads = 1 - connectors.foreach { c => + connectors.foreach { connector => // Currently we only use "SelectChannelConnector" - val connector = c.asInstanceOf[SelectChannelConnector] // Limit the max acceptor number to 8 so that we don't waste a lot of threads - connector.setAcceptors(math.min(connector.getAcceptors, 8)) + connector.setAcceptQueueSize(math.min(connector.getAcceptors, 8)) + connector.setHost(hostName) // The number of selectors always equals to the number of acceptors minThreads += connector.getAcceptors * 2 } server.setConnectors(connectors.toArray) - - val pool = new QueuedThreadPool - if (serverName.nonEmpty) { - pool.setName(serverName) - } pool.setMaxThreads(math.max(pool.getMaxThreads, minThreads)) - pool.setDaemon(true) - server.setThreadPool(pool) + val errorHandler = new ErrorHandler() errorHandler.setShowStacks(true) server.addBean(errorHandler) server.setHandler(collection) try { server.start() - (server, server.getConnectors.head.getLocalPort) + (server, httpConnector.getLocalPort) } catch { case e: Exception => server.stop() diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 6241593bba32..1aa85d60ea81 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.ui import java.net.URLDecoder import java.text.SimpleDateFormat -import java.util.{Date, Locale} +import java.util.{Date, Locale, TimeZone} import scala.util.control.NonFatal import scala.xml._ @@ -502,4 +502,7 @@ private[spark] object UIUtils extends Logging { } param } + + def getTimeZoneOffset() : Int = + TimeZone.getDefault().getOffset(System.currentTimeMillis()) / 1000 / 60 } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 07484c955013..373c26be4c5f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -23,6 +23,8 @@ import javax.servlet.http.HttpServletRequest import scala.collection.mutable.{HashMap, ListBuffer} import scala.xml._ +import org.apache.commons.lang3.StringEscapeUtils + import org.apache.spark.JobExecutionStatus import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} import org.apache.spark.ui.jobs.UIData.{ExecutorUIData, JobUIData} @@ -87,9 +89,10 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { case JobExecutionStatus.UNKNOWN => "unknown" } - // The timeline library treats contents as HTML, so we have to escape them; for the - // data-title attribute string we have to escape them twice since that's in a string. + // The timeline library treats contents as HTML, so we have to escape them. We need to add + // extra layers of escaping in order to embed this in a Javascript string literal. val escapedDesc = Utility.escape(displayJobDescription) + val jsEscapedDesc = StringEscapeUtils.escapeEcmaScript(escapedDesc) val jobEventJsonAsStr = s""" |{ @@ -99,7 +102,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { | 'end': new Date(${completionTime}), | 'content': '
    ' + | 'Status: ${status}
    ' + | 'Submitted: ${UIUtils.formatDate(new Date(submissionTime))}' + | '${ @@ -109,7 +112,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { "" } }">' + - | '${escapedDesc} (Job ${jobId})
    ' + | '${jsEscapedDesc} (Job ${jobId})' |} """.stripMargin jobEventJsonAsStr @@ -203,7 +206,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { ++ } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 645e2d2e360b..99f2bd8bc1f2 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -23,6 +23,8 @@ import javax.servlet.http.HttpServletRequest import scala.collection.mutable.{Buffer, HashMap, ListBuffer} import scala.xml.{Node, NodeSeq, Unparsed, Utility} +import org.apache.commons.lang3.StringEscapeUtils + import org.apache.spark.JobExecutionStatus import org.apache.spark.scheduler.StageInfo import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} @@ -63,9 +65,10 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val submissionTime = stage.submissionTime.get val completionTime = stage.completionTime.getOrElse(System.currentTimeMillis()) - // The timeline library treats contents as HTML, so we have to escape them; for the - // data-title attribute string we have to escape them twice since that's in a string. + // The timeline library treats contents as HTML, so we have to escape them. We need to add + // extra layers of escaping in order to embed this in a Javascript string literal. val escapedName = Utility.escape(name) + val jsEscapedName = StringEscapeUtils.escapeEcmaScript(escapedName) s""" |{ | 'className': 'stage job-timeline-object ${status}', @@ -74,7 +77,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { | 'end': new Date(${completionTime}), | 'content': '
    ' + | 'Status: ${status.toUpperCase}
    ' + | 'Submitted: ${UIUtils.formatDate(new Date(submissionTime))}' + | '${ @@ -84,7 +87,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { "" } }">' + - | '${escapedName} (Stage ${stageId}.${attemptId})
    ', + | '${jsEscapedName} (Stage ${stageId}.${attemptId})', |} """.stripMargin } @@ -176,7 +179,8 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { ++ } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 945830c8bf24..842f42b4c98d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -332,7 +332,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { new StageUIData }) stageData.numActiveTasks += 1 - stageData.taskData.put(taskInfo.taskId, new TaskUIData(taskInfo, Some(metrics))) + stageData.taskData.put(taskInfo.taskId, TaskUIData(taskInfo, Some(metrics))) } for ( activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskStart.stageId); @@ -395,9 +395,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { updateAggregateMetrics(stageData, info.executorId, m, oldMetrics) } - val taskData = stageData.taskData.getOrElseUpdate(info.taskId, new TaskUIData(info)) - taskData.taskInfo = info - taskData.metrics = taskMetrics + val taskData = stageData.taskData.getOrElseUpdate(info.taskId, TaskUIData(info, None)) + taskData.updateTaskInfo(info) + taskData.updateTaskMetrics(taskMetrics) taskData.errorMessage = errorMessage for ( @@ -425,7 +425,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageData: StageUIData, execId: String, taskMetrics: TaskMetrics, - oldMetrics: Option[TaskMetrics]) { + oldMetrics: Option[TaskMetricsUIData]) { val execSummary = stageData.executorSummary.getOrElseUpdate(execId, new ExecutorSummary) val shuffleWriteDelta = @@ -503,7 +503,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { if (!t.taskInfo.finished) { updateAggregateMetrics(stageData, executorMetricsUpdate.execId, metrics, t.metrics) // Overwrite task metrics - t.metrics = Some(metrics) + t.updateTaskMetrics(Some(metrics)) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 5d1928ac6b2c..d986a55959b8 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -26,7 +26,7 @@ import scala.xml.{Elem, Node, Unparsed} import org.apache.commons.lang3.StringEscapeUtils -import org.apache.spark.{InternalAccumulator, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo, TaskLocality} import org.apache.spark.ui._ @@ -746,7 +746,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { ++ } @@ -767,7 +768,7 @@ private[ui] object StagePage { } private[ui] def getSchedulerDelay( - info: TaskInfo, metrics: TaskMetrics, currentTime: Long): Long = { + info: TaskInfo, metrics: TaskMetricsUIData, currentTime: Long): Long = { if (info.finished) { val totalExecutionTime = info.finishTime - info.launchTime val executorOverhead = (metrics.executorDeserializeTime + diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index b454ef1b204b..d76a0e657c28 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -21,8 +21,10 @@ import scala.collection.mutable import scala.collection.mutable.HashMap import org.apache.spark.JobExecutionStatus -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ShuffleReadMetrics, ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} +import org.apache.spark.storage.{BlockId, BlockStatus} +import org.apache.spark.util.AccumulatorContext import org.apache.spark.util.collection.OpenHashSet private[spark] object UIData { @@ -105,13 +107,137 @@ private[spark] object UIData { /** * These are kept mutable and reused throughout a task's lifetime to avoid excessive reallocation. */ - class TaskUIData( - var taskInfo: TaskInfo, - var metrics: Option[TaskMetrics] = None, - var errorMessage: Option[String] = None) + class TaskUIData private( + private var _taskInfo: TaskInfo, + private var _metrics: Option[TaskMetricsUIData]) { + + var errorMessage: Option[String] = None + + def taskInfo: TaskInfo = _taskInfo + + def metrics: Option[TaskMetricsUIData] = _metrics + + def updateTaskInfo(taskInfo: TaskInfo): Unit = { + _taskInfo = TaskUIData.dropInternalAndSQLAccumulables(taskInfo) + } + + def updateTaskMetrics(metrics: Option[TaskMetrics]): Unit = { + _metrics = TaskUIData.toTaskMetricsUIData(metrics) + } + } + + object TaskUIData { + def apply(taskInfo: TaskInfo, metrics: Option[TaskMetrics]): TaskUIData = { + new TaskUIData(dropInternalAndSQLAccumulables(taskInfo), toTaskMetricsUIData(metrics)) + } + + private def toTaskMetricsUIData(metrics: Option[TaskMetrics]): Option[TaskMetricsUIData] = { + metrics.map { m => + TaskMetricsUIData( + executorDeserializeTime = m.executorDeserializeTime, + executorRunTime = m.executorRunTime, + resultSize = m.resultSize, + jvmGCTime = m.jvmGCTime, + resultSerializationTime = m.resultSerializationTime, + memoryBytesSpilled = m.memoryBytesSpilled, + diskBytesSpilled = m.diskBytesSpilled, + peakExecutionMemory = m.peakExecutionMemory, + updatedBlockStatuses = m.updatedBlockStatuses.toList, + inputMetrics = InputMetricsUIData(m.inputMetrics.bytesRead, m.inputMetrics.recordsRead), + outputMetrics = + OutputMetricsUIData(m.outputMetrics.bytesWritten, m.outputMetrics.recordsWritten), + shuffleReadMetrics = ShuffleReadMetricsUIData(m.shuffleReadMetrics), + shuffleWriteMetrics = ShuffleWriteMetricsUIData(m.shuffleWriteMetrics)) + } + } + + /** + * We don't need to store internal or SQL accumulables as their values will be shown in other + * places, so drop them to reduce the memory usage. + */ + private[spark] def dropInternalAndSQLAccumulables(taskInfo: TaskInfo): TaskInfo = { + val newTaskInfo = new TaskInfo( + taskId = taskInfo.taskId, + index = taskInfo.index, + attemptNumber = taskInfo.attemptNumber, + launchTime = taskInfo.launchTime, + executorId = taskInfo.executorId, + host = taskInfo.host, + taskLocality = taskInfo.taskLocality, + speculative = taskInfo.speculative + ) + newTaskInfo.gettingResultTime = taskInfo.gettingResultTime + newTaskInfo.accumulables ++= taskInfo.accumulables.filter { + accum => !accum.internal && accum.metadata != Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER) + } + newTaskInfo.finishTime = taskInfo.finishTime + newTaskInfo.failed = taskInfo.failed + newTaskInfo + } + } class ExecutorUIData( val startTime: Long, var finishTime: Option[Long] = None, var finishReason: Option[String] = None) + + case class TaskMetricsUIData( + executorDeserializeTime: Long, + executorRunTime: Long, + resultSize: Long, + jvmGCTime: Long, + resultSerializationTime: Long, + memoryBytesSpilled: Long, + diskBytesSpilled: Long, + peakExecutionMemory: Long, + updatedBlockStatuses: Seq[(BlockId, BlockStatus)], + inputMetrics: InputMetricsUIData, + outputMetrics: OutputMetricsUIData, + shuffleReadMetrics: ShuffleReadMetricsUIData, + shuffleWriteMetrics: ShuffleWriteMetricsUIData) + + case class InputMetricsUIData(bytesRead: Long, recordsRead: Long) + + case class OutputMetricsUIData(bytesWritten: Long, recordsWritten: Long) + + case class ShuffleReadMetricsUIData( + remoteBlocksFetched: Long, + localBlocksFetched: Long, + remoteBytesRead: Long, + localBytesRead: Long, + fetchWaitTime: Long, + recordsRead: Long, + totalBytesRead: Long, + totalBlocksFetched: Long) + + object ShuffleReadMetricsUIData { + def apply(metrics: ShuffleReadMetrics): ShuffleReadMetricsUIData = { + new ShuffleReadMetricsUIData( + remoteBlocksFetched = metrics.remoteBlocksFetched, + localBlocksFetched = metrics.localBlocksFetched, + remoteBytesRead = metrics.remoteBytesRead, + localBytesRead = metrics.localBytesRead, + fetchWaitTime = metrics.fetchWaitTime, + recordsRead = metrics.recordsRead, + totalBytesRead = metrics.totalBytesRead, + totalBlocksFetched = metrics.totalBlocksFetched + ) + } + } + + case class ShuffleWriteMetricsUIData( + bytesWritten: Long, + recordsWritten: Long, + writeTime: Long) + + object ShuffleWriteMetricsUIData { + def apply(metrics: ShuffleWriteMetrics): ShuffleWriteMetricsUIData = { + new ShuffleWriteMetricsUIData( + bytesWritten = metrics.bytesWritten, + recordsWritten = metrics.recordsWritten, + writeTime = metrics.writeTime + ) + } + } + } diff --git a/core/src/main/scala/org/apache/spark/NewAccumulator.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala similarity index 56% rename from core/src/main/scala/org/apache/spark/NewAccumulator.scala rename to core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index 1571e15b76ac..13cb6a28c346 100644 --- a/core/src/main/scala/org/apache/spark/NewAccumulator.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -15,18 +15,15 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.util import java.{lang => jl} import java.io.ObjectInputStream import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicLong -import javax.annotation.concurrent.GuardedBy - -import scala.collection.JavaConverters._ +import org.apache.spark.{InternalAccumulator, SparkContext, TaskContext} import org.apache.spark.scheduler.AccumulableInfo -import org.apache.spark.util.Utils private[spark] case class AccumulatorMetadata( @@ -39,7 +36,7 @@ private[spark] case class AccumulatorMetadata( * The base class for accumulators, that can accumulate inputs of type `IN`, and produce output of * type `OUT`. */ -abstract class NewAccumulator[IN, OUT] extends Serializable { +abstract class AccumulatorV2[IN, OUT] extends Serializable { private[spark] var metadata: AccumulatorMetadata = _ private[this] var atDriverSide = true @@ -95,7 +92,7 @@ abstract class NewAccumulator[IN, OUT] extends Serializable { } /** - * Creates an [[AccumulableInfo]] representation of this [[NewAccumulator]] with the provided + * Creates an [[AccumulableInfo]] representation of this [[AccumulatorV2]] with the provided * values. */ private[spark] def toInfo(update: Option[Any], value: Option[Any]): AccumulableInfo = { @@ -106,16 +103,31 @@ abstract class NewAccumulator[IN, OUT] extends Serializable { final private[spark] def isAtDriverSide: Boolean = atDriverSide /** - * Tells if this accumulator is zero value or not. e.g. for a counter accumulator, 0 is zero + * Returns if this accumulator is zero value or not. e.g. for a counter accumulator, 0 is zero * value; for a list accumulator, Nil is zero value. */ - def isZero(): Boolean + def isZero: Boolean /** * Creates a new copy of this accumulator, which is zero value. i.e. call `isZero` on the copy * must return true. */ - def copyAndReset(): NewAccumulator[IN, OUT] + def copyAndReset(): AccumulatorV2[IN, OUT] = { + val copyAcc = copy() + copyAcc.reset() + copyAcc + } + + /** + * Creates a new copy of this accumulator. + */ + def copy(): AccumulatorV2[IN, OUT] + + /** + * Resets this accumulator, which is zero value. i.e. call `isZero` must + * return true. + */ + def reset(): Unit /** * Takes the inputs and accumulates. e.g. it can be a simple `+=` for counter accumulator. @@ -126,26 +138,12 @@ abstract class NewAccumulator[IN, OUT] extends Serializable { * Merges another same-type accumulator into this one and update its state, i.e. this should be * merge-in-place. */ - def merge(other: NewAccumulator[IN, OUT]): Unit - - /** - * Access this accumulator's current value; only allowed on driver. - */ - final def value: OUT = { - if (atDriverSide) { - localValue - } else { - throw new UnsupportedOperationException("Can't read accumulator value in task") - } - } + def merge(other: AccumulatorV2[IN, OUT]): Unit /** - * Defines the current value of this accumulator. - * - * This is NOT the global value of the accumulator. To get the global value after a - * completed operation on the dataset, call `value`. + * Defines the current value of this accumulator */ - def localValue: OUT + def value: OUT // Called by Java when serializing an object final protected def writeReplace(): Any = { @@ -154,10 +152,10 @@ abstract class NewAccumulator[IN, OUT] extends Serializable { throw new UnsupportedOperationException( "Accumulator must be registered before send to executor") } - val copy = copyAndReset() - assert(copy.isZero(), "copyAndReset must return a zero value copy") - copy.metadata = metadata - copy + val copyAcc = copyAndReset() + assert(copyAcc.isZero, "copyAndReset must return a zero value copy") + copyAcc.metadata = metadata + copyAcc } else { this } @@ -185,12 +183,15 @@ abstract class NewAccumulator[IN, OUT] extends Serializable { if (metadata == null) { "Un-registered Accumulator: " + getClass.getSimpleName } else { - getClass.getSimpleName + s"(id: $id, name: $name, value: $localValue)" + getClass.getSimpleName + s"(id: $id, name: $name, value: $value)" } } } +/** + * An internal class used to track accumulators by Spark itself. + */ private[spark] object AccumulatorContext { /** @@ -199,44 +200,45 @@ private[spark] object AccumulatorContext { * once the RDDs and user-code that reference them are cleaned up. * TODO: Don't use a global map; these should be tied to a SparkContext (SPARK-13051). */ - private val originals = new ConcurrentHashMap[Long, jl.ref.WeakReference[NewAccumulator[_, _]]] + private val originals = new ConcurrentHashMap[Long, jl.ref.WeakReference[AccumulatorV2[_, _]]] private[this] val nextId = new AtomicLong(0L) /** - * Return a globally unique ID for a new [[Accumulator]]. - * Note: Once you copy the [[Accumulator]] the ID is no longer unique. + * Returns a globally unique ID for a new [[AccumulatorV2]]. + * Note: Once you copy the [[AccumulatorV2]] the ID is no longer unique. */ def newId(): Long = nextId.getAndIncrement + /** Returns the number of accumulators registered. Used in testing. */ def numAccums: Int = originals.size /** - * Register an [[Accumulator]] created on the driver such that it can be used on the executors. + * Registers an [[AccumulatorV2]] created on the driver such that it can be used on the executors. * * All accumulators registered here can later be used as a container for accumulating partial * values across multiple tasks. This is what [[org.apache.spark.scheduler.DAGScheduler]] does. * Note: if an accumulator is registered here, it should also be registered with the active * context cleaner for cleanup so as to avoid memory leaks. * - * If an [[Accumulator]] with the same ID was already registered, this does nothing instead + * If an [[AccumulatorV2]] with the same ID was already registered, this does nothing instead * of overwriting it. We will never register same accumulator twice, this is just a sanity check. */ - def register(a: NewAccumulator[_, _]): Unit = { - originals.putIfAbsent(a.id, new jl.ref.WeakReference[NewAccumulator[_, _]](a)) + def register(a: AccumulatorV2[_, _]): Unit = { + originals.putIfAbsent(a.id, new jl.ref.WeakReference[AccumulatorV2[_, _]](a)) } /** - * Unregister the [[Accumulator]] with the given ID, if any. + * Unregisters the [[AccumulatorV2]] with the given ID, if any. */ def remove(id: Long): Unit = { originals.remove(id) } /** - * Return the [[Accumulator]] registered with the given ID, if any. + * Returns the [[AccumulatorV2]] registered with the given ID, if any. */ - def get(id: Long): Option[NewAccumulator[_, _]] = { + def get(id: Long): Option[AccumulatorV2[_, _]] = { Option(originals.get(id)).map { ref => // Since we are storing weak references, we must check whether the underlying data is valid. val acc = ref.get @@ -248,118 +250,194 @@ private[spark] object AccumulatorContext { } /** - * Clear all registered [[Accumulator]]s. For testing only. + * Clears all registered [[AccumulatorV2]]s. For testing only. */ def clear(): Unit = { originals.clear() } -} - - -class LongAccumulator extends NewAccumulator[jl.Long, jl.Long] { - private[this] var _sum = 0L - override def isZero(): Boolean = _sum == 0 - - override def copyAndReset(): LongAccumulator = new LongAccumulator + // Identifier for distinguishing SQL metrics from other accumulators + private[spark] val SQL_ACCUM_IDENTIFIER = "sql" +} - override def add(v: jl.Long): Unit = _sum += v - def add(v: Long): Unit = _sum += v +/** + * An [[AccumulatorV2 accumulator]] for computing sum, count, and averages for 64-bit integers. + * + * @since 2.0.0 + */ +class LongAccumulator extends AccumulatorV2[jl.Long, jl.Long] { + private var _sum = 0L + private var _count = 0L - def sum: Long = _sum + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + override def isZero: Boolean = _sum == 0L && _count == 0 - override def merge(other: NewAccumulator[jl.Long, jl.Long]): Unit = other match { - case o: LongAccumulator => _sum += o.sum - case _ => throw new UnsupportedOperationException( - s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + override def copy(): LongAccumulator = { + val newAcc = new LongAccumulator + newAcc._count = this._count + newAcc._sum = this._sum + newAcc } - private[spark] def setValue(newValue: Long): Unit = _sum = newValue - - override def localValue: jl.Long = _sum -} - - -class DoubleAccumulator extends NewAccumulator[jl.Double, jl.Double] { - private[this] var _sum = 0.0 + override def reset(): Unit = { + _sum = 0L + _count = 0L + } - override def isZero(): Boolean = _sum == 0.0 + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + override def add(v: jl.Long): Unit = { + _sum += v + _count += 1 + } - override def copyAndReset(): DoubleAccumulator = new DoubleAccumulator + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + def add(v: Long): Unit = { + _sum += v + _count += 1 + } - override def add(v: jl.Double): Unit = _sum += v + /** + * Returns the number of elements added to the accumulator. + * @since 2.0.0 + */ + def count: Long = _count - def add(v: Double): Unit = _sum += v + /** + * Returns the sum of elements added to the accumulator. + * @since 2.0.0 + */ + def sum: Long = _sum - def sum: Double = _sum + /** + * Returns the average of elements added to the accumulator. + * @since 2.0.0 + */ + def avg: Double = _sum.toDouble / _count - override def merge(other: NewAccumulator[jl.Double, jl.Double]): Unit = other match { - case o: DoubleAccumulator => _sum += o.sum - case _ => throw new UnsupportedOperationException( - s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + override def merge(other: AccumulatorV2[jl.Long, jl.Long]): Unit = other match { + case o: LongAccumulator => + _sum += o.sum + _count += o.count + case _ => + throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } - private[spark] def setValue(newValue: Double): Unit = _sum = newValue + private[spark] def setValue(newValue: Long): Unit = _sum = newValue - override def localValue: jl.Double = _sum + override def value: jl.Long = _sum } -class AverageAccumulator extends NewAccumulator[jl.Double, jl.Double] { - private[this] var _sum = 0.0 - private[this] var _count = 0L +/** + * An [[AccumulatorV2 accumulator]] for computing sum, count, and averages for double precision + * floating numbers. + * + * @since 2.0.0 + */ +class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] { + private var _sum = 0.0 + private var _count = 0L + + override def isZero: Boolean = _sum == 0.0 && _count == 0 - override def isZero(): Boolean = _sum == 0.0 && _count == 0 + override def copy(): DoubleAccumulator = { + val newAcc = new DoubleAccumulator + newAcc._count = this._count + newAcc._sum = this._sum + newAcc + } - override def copyAndReset(): AverageAccumulator = new AverageAccumulator + override def reset(): Unit = { + _sum = 0.0 + _count = 0L + } + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ override def add(v: jl.Double): Unit = { _sum += v _count += 1 } - def add(d: Double): Unit = { - _sum += d + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + def add(v: Double): Unit = { + _sum += v _count += 1 } - override def merge(other: NewAccumulator[jl.Double, jl.Double]): Unit = other match { - case o: AverageAccumulator => + /** + * Returns the number of elements added to the accumulator. + * @since 2.0.0 + */ + def count: Long = _count + + /** + * Returns the sum of elements added to the accumulator. + * @since 2.0.0 + */ + def sum: Double = _sum + + /** + * Returns the average of elements added to the accumulator. + * @since 2.0.0 + */ + def avg: Double = _sum / _count + + override def merge(other: AccumulatorV2[jl.Double, jl.Double]): Unit = other match { + case o: DoubleAccumulator => _sum += o.sum _count += o.count - case _ => throw new UnsupportedOperationException( - s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") - } - - override def localValue: jl.Double = if (_count == 0) { - Double.NaN - } else { - _sum / _count + case _ => + throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } - def sum: Double = _sum + private[spark] def setValue(newValue: Double): Unit = _sum = newValue - def count: Long = _count + override def value: jl.Double = _sum } -class ListAccumulator[T] extends NewAccumulator[T, java.util.List[T]] { - private[this] val _list: java.util.List[T] = new java.util.ArrayList[T] +class ListAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { + private val _list: java.util.List[T] = new java.util.ArrayList[T] - override def isZero(): Boolean = _list.isEmpty + override def isZero: Boolean = _list.isEmpty override def copyAndReset(): ListAccumulator[T] = new ListAccumulator + override def copy(): ListAccumulator[T] = { + val newAcc = new ListAccumulator[T] + newAcc._list.addAll(_list) + newAcc + } + + override def reset(): Unit = _list.clear() + override def add(v: T): Unit = _list.add(v) - override def merge(other: NewAccumulator[T, java.util.List[T]]): Unit = other match { - case o: ListAccumulator[T] => _list.addAll(o.localValue) + override def merge(other: AccumulatorV2[T, java.util.List[T]]): Unit = other match { + case o: ListAccumulator[T] => _list.addAll(o.value) case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } - override def localValue: java.util.List[T] = java.util.Collections.unmodifiableList(_list) + override def value: java.util.List[T] = java.util.Collections.unmodifiableList(_list) private[spark] def setValue(newValue: java.util.List[T]): Unit = { _list.clear() @@ -370,24 +448,28 @@ class ListAccumulator[T] extends NewAccumulator[T, java.util.List[T]] { class LegacyAccumulatorWrapper[R, T]( initialValue: R, - param: org.apache.spark.AccumulableParam[R, T]) extends NewAccumulator[T, R] { + param: org.apache.spark.AccumulableParam[R, T]) extends AccumulatorV2[T, R] { private[spark] var _value = initialValue // Current value on driver - override def isZero(): Boolean = _value == param.zero(initialValue) + override def isZero: Boolean = _value == param.zero(initialValue) - override def copyAndReset(): LegacyAccumulatorWrapper[R, T] = { + override def copy(): LegacyAccumulatorWrapper[R, T] = { val acc = new LegacyAccumulatorWrapper(initialValue, param) - acc._value = param.zero(initialValue) + acc._value = _value acc } + override def reset(): Unit = { + _value = param.zero(initialValue) + } + override def add(v: T): Unit = _value = param.addAccumulator(_value, v) - override def merge(other: NewAccumulator[T, R]): Unit = other match { - case o: LegacyAccumulatorWrapper[R, T] => _value = param.addInPlace(_value, o.localValue) + override def merge(other: AccumulatorV2[T, R]): Unit = other match { + case o: LegacyAccumulatorWrapper[R, T] => _value = param.addInPlace(_value, o.value) case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } - override def localValue: R = _value + override def value: R = _value } diff --git a/core/src/main/scala/org/apache/spark/util/Benchmark.scala b/core/src/main/scala/org/apache/spark/util/Benchmark.scala index 1fc0ad7a4d6d..0c685b1918a5 100644 --- a/core/src/main/scala/org/apache/spark/util/Benchmark.scala +++ b/core/src/main/scala/org/apache/spark/util/Benchmark.scala @@ -38,7 +38,7 @@ import org.apache.commons.lang3.SystemUtils private[spark] class Benchmark( name: String, valuesPerIteration: Long, - iters: Int = 5, + defaultNumIters: Int = 5, outputPerIteration: Boolean = false) { val benchmarks = mutable.ArrayBuffer.empty[Benchmark.Case] @@ -46,8 +46,8 @@ private[spark] class Benchmark( * Adds a case to run when run() is called. The given function will be run for several * iterations to collect timing statistics. */ - def addCase(name: String)(f: Int => Unit): Unit = { - addTimerCase(name) { timer => + def addCase(name: String, numIters: Int = 0)(f: Int => Unit): Unit = { + addTimerCase(name, numIters) { timer => timer.startTiming() f(timer.iteration) timer.stopTiming() @@ -59,8 +59,8 @@ private[spark] class Benchmark( * until timer.startTiming() is called within the given function. The corresponding * timer.stopTiming() method must be called before the function returns. */ - def addTimerCase(name: String)(f: Benchmark.Timer => Unit): Unit = { - benchmarks += Benchmark.Case(name, f) + def addTimerCase(name: String, numIters: Int = 0)(f: Benchmark.Timer => Unit): Unit = { + benchmarks += Benchmark.Case(name, f, if (numIters == 0) defaultNumIters else numIters) } /** @@ -75,7 +75,7 @@ private[spark] class Benchmark( val results = benchmarks.map { c => println(" Running case: " + c.name) - Benchmark.measure(valuesPerIteration, iters, outputPerIteration)(c.fn) + Benchmark.measure(valuesPerIteration, c.numIters, outputPerIteration)(c.fn) } println @@ -83,12 +83,11 @@ private[spark] class Benchmark( // The results are going to be processor specific so it is useful to include that. println(Benchmark.getJVMOSInfo()) println(Benchmark.getProcessorName()) - printf("%-35s %16s %12s %13s %10s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", + printf("%-40s %16s %12s %13s %10s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", "Per Row(ns)", "Relative") - println("-----------------------------------------------------------------------------------" + - "--------") + println("-" * 96) results.zip(benchmarks).foreach { case (result, benchmark) => - printf("%-35s %16s %12s %13s %10s\n", + printf("%-40s %16s %12s %13s %10s\n", benchmark.name, "%5.0f / %4.0f" format (result.bestMs, result.avgMs), "%10.1f" format result.bestRate, @@ -128,7 +127,7 @@ private[spark] object Benchmark { } } - case class Case(name: String, fn: Timer => Unit) + case class Case(name: String, fn: Timer => Unit, numIters: Int) case class Result(avgMs: Double, bestRate: Double, bestMs: Double) /** diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index aeab71d9df60..18547d459eb5 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -841,7 +841,7 @@ private[spark] object JsonProtocol { val accumUpdates = Utils.jsonOption(json \ "Accumulator Updates") .map(_.extract[List[JValue]].map(accumulableInfoFromJson)) .getOrElse(taskMetricsFromJson(json \ "Metrics").accumulators().map(acc => { - acc.toInfo(Some(acc.localValue), None) + acc.toInfo(Some(acc.value), None) })) ExceptionFailure(className, description, stackTrace, fullStackTrace, None, accumUpdates) case `taskResultLost` => TaskResultLost diff --git a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala index 0a3180da8798..034826c57ef1 100644 --- a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala @@ -19,7 +19,6 @@ package org.apache.spark.util import java.net.{URL, URLClassLoader} import java.util.Enumeration -import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ @@ -48,32 +47,12 @@ private[spark] class ChildFirstURLClassLoader(urls: Array[URL], parent: ClassLoa private val parentClassLoader = new ParentClassLoader(parent) - /** - * Used to implement fine-grained class loading locks similar to what is done by Java 7. This - * prevents deadlock issues when using non-hierarchical class loaders. - * - * Note that due to some issues with implementing class loaders in - * Scala, Java 7's `ClassLoader.registerAsParallelCapable` method is not called. - */ - private val locks = new ConcurrentHashMap[String, Object]() - override def loadClass(name: String, resolve: Boolean): Class[_] = { - var lock = locks.get(name) - if (lock == null) { - val newLock = new Object() - lock = locks.putIfAbsent(name, newLock) - if (lock == null) { - lock = newLock - } - } - - lock.synchronized { - try { - super.loadClass(name, resolve) - } catch { - case e: ClassNotFoundException => - parentClassLoader.loadClass(name, resolve) - } + try { + super.loadClass(name, resolve) + } catch { + case e: ClassNotFoundException => + parentClassLoader.loadClass(name, resolve) } } diff --git a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala index bd26bfd848ff..93ac67e5db0d 100644 --- a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala +++ b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala @@ -170,9 +170,7 @@ private [util] class SparkShutdownHookManager { @volatile private var shuttingDown = false /** - * Install a hook to run at shutdown and run all registered hooks in order. Hadoop 1.x does not - * have `ShutdownHookManager`, so in that case we just use the JVM's `Runtime` object and hope for - * the best. + * Install a hook to run at shutdown and run all registered hooks in order. */ def install(): Unit = { val hookTask = new Runnable() { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index ea49991493fd..a8bb0002a7b2 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2181,6 +2181,25 @@ private[spark] object Utils extends Logging { .getOrElse(UserGroupInformation.getCurrentUser().getShortUserName()) } + val EMPTY_USER_GROUPS = Set[String]() + + // Returns the groups to which the current user belongs. + def getCurrentUserGroups(sparkConf: SparkConf, username: String): Set[String] = { + val groupProviderClassName = sparkConf.get("spark.user.groups.mapping", + "org.apache.spark.security.ShellBasedGroupsMappingProvider") + if (groupProviderClassName != "") { + try { + val groupMappingServiceProvider = classForName(groupProviderClassName).newInstance. + asInstanceOf[org.apache.spark.security.GroupMappingServiceProvider] + val currentUserGroups = groupMappingServiceProvider.getGroups(username) + return currentUserGroups + } catch { + case e: Exception => logError(s"Error getting groups for user=$username", e) + } + } + EMPTY_USER_GROUPS + } + /** * Split the comma delimited string of master URLs into a list. * For instance, "spark://abc,def" becomes [spark://abc, spark://def]. diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala index aee6399eb0c8..8183f825592c 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -41,7 +41,7 @@ private[spark] abstract class Spillable[C](taskMemoryManager: TaskMemoryManager) protected def forceSpill(): Boolean // Number of elements read from input since last spill - @volatile protected def elementsRead: Long = _elementsRead + protected def elementsRead: Long = _elementsRead // Called by subclasses every time a record is read // It's used for checking spilling frequency @@ -83,7 +83,7 @@ private[spark] abstract class Spillable[C](taskMemoryManager: TaskMemoryManager) if (elementsRead % 32 == 0 && currentMemory >= myMemoryThreshold) { // Claim up to double our current memory from the shuffle memory pool val amountToRequest = 2 * currentMemory - myMemoryThreshold - val granted = acquireOnHeapMemory(amountToRequest) + val granted = acquireMemory(amountToRequest) myMemoryThreshold += granted // If we were granted too little memory to grow further (either tryToAcquire returned 0, // or we already had more memory than myMemoryThreshold), spill the current collection @@ -112,7 +112,6 @@ private[spark] abstract class Spillable[C](taskMemoryManager: TaskMemoryManager) if (!isSpilled) { 0L } else { - _elementsRead = 0 val freeMemory = myMemoryThreshold - initialMemoryThreshold _memoryBytesSpilled += freeMemory releaseMemory() @@ -132,7 +131,7 @@ private[spark] abstract class Spillable[C](taskMemoryManager: TaskMemoryManager) * Release our memory back to the execution pool so that other tasks can grab it. */ def releaseMemory(): Unit = { - freeOnHeapMemory(myMemoryThreshold - initialMemoryThreshold) + freeMemory(myMemoryThreshold - initialMemoryThreshold) myMemoryThreshold = initialMemoryThreshold } diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala index 6e80db2f51f9..5c4238c0381a 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala @@ -113,7 +113,7 @@ private[spark] class SizeBasedRollingPolicy( /** Should rollover if the next set of bytes is going to exceed the size limit */ def shouldRollover(bytesToBeWritten: Long): Boolean = { - logInfo(s"$bytesToBeWritten + $bytesWrittenSinceRollover > $rolloverSizeBytes") + logDebug(s"$bytesToBeWritten + $bytesWrittenSinceRollover > $rolloverSizeBytes") bytesToBeWritten + bytesWrittenSinceRollover > rolloverSizeBytes } diff --git a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java index 3e47bfc274cb..8ca54b24d82e 100644 --- a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -94,7 +94,7 @@ public void testChildProcLauncher() throws Exception { SparkLauncher launcher = new SparkLauncher(env) .setSparkHome(System.getProperty("spark.test.home")) .setMaster("local") - .setAppResource("spark-internal") + .setAppResource(SparkLauncher.NO_RESOURCE) .addSparkArg(opts.CONF, String.format("%s=-Dfoo=ShouldBeOverriddenBelow", SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS)) .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java index 127789b632b4..ad755529dec6 100644 --- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java +++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java @@ -34,7 +34,8 @@ public void leakedPageMemoryIsDetected() { Long.MAX_VALUE, 1), 0); - manager.allocatePage(4096, null); // leak memory + final MemoryConsumer c = new TestMemoryConsumer(manager); + manager.allocatePage(4096, c); // leak memory Assert.assertEquals(4096, manager.getMemoryConsumptionForThisTask()); Assert.assertEquals(4096, manager.cleanUpAllAllocatedMemory()); } @@ -45,7 +46,8 @@ public void encodePageNumberAndOffsetOffHeap() { .set("spark.memory.offHeap.enabled", "true") .set("spark.memory.offHeap.size", "1000"); final TaskMemoryManager manager = new TaskMemoryManager(new TestMemoryManager(conf), 0); - final MemoryBlock dataPage = manager.allocatePage(256, null); + final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.OFF_HEAP); + final MemoryBlock dataPage = manager.allocatePage(256, c); // In off-heap mode, an offset is an absolute address that may require more than 51 bits to // encode. This test exercises that corner-case: final long offset = ((1L << TaskMemoryManager.OFFSET_BITS) + 10); @@ -58,7 +60,8 @@ public void encodePageNumberAndOffsetOffHeap() { public void encodePageNumberAndOffsetOnHeap() { final TaskMemoryManager manager = new TaskMemoryManager( new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); - final MemoryBlock dataPage = manager.allocatePage(256, null); + final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP); + final MemoryBlock dataPage = manager.allocatePage(256, c); final long encodedAddress = manager.encodePageNumberAndOffset(dataPage, 64); Assert.assertEquals(dataPage.getBaseObject(), manager.getPage(encodedAddress)); Assert.assertEquals(64, manager.getOffsetInPage(encodedAddress)); @@ -106,6 +109,25 @@ public void cooperativeSpilling() { Assert.assertEquals(0, manager.cleanUpAllAllocatedMemory()); } + @Test + public void shouldNotForceSpillingInDifferentModes() { + final TestMemoryManager memoryManager = new TestMemoryManager(new SparkConf()); + memoryManager.limit(100); + final TaskMemoryManager manager = new TaskMemoryManager(memoryManager, 0); + + TestMemoryConsumer c1 = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP); + TestMemoryConsumer c2 = new TestMemoryConsumer(manager, MemoryMode.OFF_HEAP); + c1.use(80); + Assert.assertEquals(80, c1.getUsed()); + c2.use(80); + Assert.assertEquals(20, c2.getUsed()); // not enough memory + Assert.assertEquals(80, c1.getUsed()); // not spilled + + c2.use(10); + Assert.assertEquals(10, c2.getUsed()); // spilled + Assert.assertEquals(80, c1.getUsed()); // not spilled + } + @Test public void offHeapConfigurationBackwardsCompatibility() { // Tests backwards-compatibility with the old `spark.unsafe.offHeap` configuration, which diff --git a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java index e6e16fff8040..db91329c94cb 100644 --- a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java +++ b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java @@ -20,8 +20,11 @@ import java.io.IOException; public class TestMemoryConsumer extends MemoryConsumer { + public TestMemoryConsumer(TaskMemoryManager memoryManager, MemoryMode mode) { + super(memoryManager, 1024L, mode); + } public TestMemoryConsumer(TaskMemoryManager memoryManager) { - super(memoryManager); + this(memoryManager, MemoryMode.ON_HEAP); } @Override @@ -32,19 +35,13 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { } void use(long size) { - long got = taskMemoryManager.acquireExecutionMemory( - size, - taskMemoryManager.tungstenMemoryMode, - this); + long got = taskMemoryManager.acquireExecutionMemory(size, this); used += got; } void free(long size) { used -= size; - taskMemoryManager.releaseExecutionMemory( - size, - taskMemoryManager.tungstenMemoryMode, - this); + taskMemoryManager.releaseExecutionMemory(size, this); } } diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java index fe5abc5c2304..354efe18dbde 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java @@ -22,8 +22,7 @@ import org.junit.Test; import org.apache.spark.SparkConf; -import org.apache.spark.memory.TestMemoryManager; -import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.memory.*; import org.apache.spark.unsafe.memory.MemoryBlock; import static org.apache.spark.shuffle.sort.PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES; @@ -38,8 +37,9 @@ public void heap() throws IOException { final SparkConf conf = new SparkConf().set("spark.memory.offHeap.enabled", "false"); final TaskMemoryManager memoryManager = new TaskMemoryManager(new TestMemoryManager(conf), 0); - final MemoryBlock page0 = memoryManager.allocatePage(128, null); - final MemoryBlock page1 = memoryManager.allocatePage(128, null); + final MemoryConsumer c = new TestMemoryConsumer(memoryManager, MemoryMode.ON_HEAP); + final MemoryBlock page0 = memoryManager.allocatePage(128, c); + final MemoryBlock page1 = memoryManager.allocatePage(128, c); final long addressInPage1 = memoryManager.encodePageNumberAndOffset(page1, page1.getBaseOffset() + 42); PackedRecordPointer packedPointer = new PackedRecordPointer(); @@ -59,8 +59,9 @@ public void offHeap() throws IOException { .set("spark.memory.offHeap.size", "10000"); final TaskMemoryManager memoryManager = new TaskMemoryManager(new TestMemoryManager(conf), 0); - final MemoryBlock page0 = memoryManager.allocatePage(128, null); - final MemoryBlock page1 = memoryManager.allocatePage(128, null); + final MemoryConsumer c = new TestMemoryConsumer(memoryManager, MemoryMode.OFF_HEAP); + final MemoryBlock page0 = memoryManager.allocatePage(128, c); + final MemoryBlock page1 = memoryManager.allocatePage(128, c); final long addressInPage1 = memoryManager.encodePageNumberAndOffset(page1, page1.getBaseOffset() + 42); PackedRecordPointer packedPointer = new PackedRecordPointer(); diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java index 278a827644db..694352ee2af4 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java @@ -26,6 +26,7 @@ import org.apache.spark.HashPartitioner; import org.apache.spark.SparkConf; +import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.memory.TestMemoryConsumer; import org.apache.spark.memory.TestMemoryManager; @@ -71,7 +72,8 @@ public void testBasicSorting() throws Exception { final SparkConf conf = new SparkConf().set("spark.memory.offHeap.enabled", "false"); final TaskMemoryManager memoryManager = new TaskMemoryManager(new TestMemoryManager(conf), 0); - final MemoryBlock dataPage = memoryManager.allocatePage(2048, null); + final MemoryConsumer c = new TestMemoryConsumer(memoryManager); + final MemoryBlock dataPage = memoryManager.allocatePage(2048, c); final Object baseObject = dataPage.getBaseObject(); final ShuffleInMemorySorter sorter = new ShuffleInMemorySorter( consumer, 4, shouldUseRadixSort()); diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java index 60a40cc17211..2cae4beb4c77 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java @@ -49,6 +49,7 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.util.Utils; +import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.junit.Assert.*; import static org.mockito.Answers.RETURNS_SMART_NULLS; @@ -225,6 +226,25 @@ public void testSortingEmptyArrays() throws Exception { assertSpillFilesWereCleanedUp(); } + @Test + public void testSortTimeMetric() throws Exception { + final UnsafeExternalSorter sorter = newSorter(); + long prevSortTime = sorter.getSortTimeNanos(); + assertEquals(prevSortTime, 0); + + sorter.insertRecord(null, 0, 0, 0); + sorter.spill(); + assertThat(sorter.getSortTimeNanos(), greaterThan(prevSortTime)); + prevSortTime = sorter.getSortTimeNanos(); + + sorter.spill(); // no sort needed + assertEquals(sorter.getSortTimeNanos(), prevSortTime); + + sorter.insertRecord(null, 0, 0, 0); + UnsafeSorterIterator iter = sorter.getSortedIterator(); + assertThat(sorter.getSortTimeNanos(), greaterThan(prevSortTime)); + } + @Test public void spillingOccursInResponseToMemoryPressure() throws Exception { final UnsafeExternalSorter sorter = newSorter(); diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java index 4a2f65a0ed2b..383c5b3b0884 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java @@ -78,7 +78,7 @@ public void testSortingOnlyByIntegerPrefix() throws Exception { final TaskMemoryManager memoryManager = new TaskMemoryManager( new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); final TestMemoryConsumer consumer = new TestMemoryConsumer(memoryManager); - final MemoryBlock dataPage = memoryManager.allocatePage(2048, null); + final MemoryBlock dataPage = memoryManager.allocatePage(2048, consumer); final Object baseObject = dataPage.getBaseObject(); // Write the records into the data page: long position = dataPage.getBaseOffset(); diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 9c90049715dd..6cbd5ae5d428 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -28,9 +28,10 @@ import scala.util.control.NonFatal import org.scalatest.Matchers import org.scalatest.exceptions.TestFailedException -import org.apache.spark.AccumulatorParam.{ListAccumulatorParam, StringAccumulatorParam} +import org.apache.spark.AccumulatorParam.StringAccumulatorParam import org.apache.spark.scheduler._ import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, AccumulatorV2, LongAccumulator} class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContext { @@ -69,7 +70,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex // serialize and de-serialize it, to simulate sending accumulator to executor. val acc2 = ser.deserialize[LongAccumulator](ser.serialize(acc)) // value is reset on the executors - assert(acc2.localValue == 0) + assert(acc2.value == 0) assert(!acc2.isAtDriverSide) acc2.add(10) @@ -234,21 +235,6 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex acc.merge("kindness") assert(acc.value === "kindness") } - - test("list accumulator param") { - val acc = new Accumulator(Seq.empty[Int], new ListAccumulatorParam[Int], Some("numbers")) - assert(acc.value === Seq.empty[Int]) - acc.add(Seq(1, 2)) - assert(acc.value === Seq(1, 2)) - acc += Seq(3, 4) - assert(acc.value === Seq(1, 2, 3, 4)) - acc ++= Seq(5, 6) - assert(acc.value === Seq(1, 2, 3, 4, 5, 6)) - acc.merge(Seq(7, 8)) - assert(acc.value === Seq(1, 2, 3, 4, 5, 6, 7, 8)) - acc.setValue(Seq(9, 10)) - assert(acc.value === Seq(9, 10)) - } } private[spark] object AccumulatorSuite { @@ -273,7 +259,7 @@ private[spark] object AccumulatorSuite { * Make an [[AccumulableInfo]] out of an [[Accumulable]] with the intent to use the * info as an accumulator update. */ - def makeInfo(a: NewAccumulator[_, _]): AccumulableInfo = a.toInfo(Some(a.localValue), None) + def makeInfo(a: AccumulatorV2[_, _]): AccumulableInfo = a.toInfo(Some(a.value), None) /** * Run one or more Spark jobs and verify that in at least one job the peak execution memory diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index a0086e18435b..0be25e9f893b 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -196,7 +196,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex val blockIds = data.partitions.indices.map(index => RDDBlockId(data.id, index)).toArray val blockId = blockIds(0) val blockManager = SparkEnv.get.blockManager - val blockTransfer = SparkEnv.get.blockTransferService + val blockTransfer = blockManager.blockTransferService val serializerManager = SparkEnv.get.serializerManager blockManager.master.getLocations(blockId).foreach { cmId => val bytes = blockTransfer.fetchBlockSync(cmId.host, cmId.port, cmId.executorId, diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index 3def8b0b1850..333c23bdaf6d 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark import java.io.{IOException, NotSerializableException, ObjectInputStream} +import org.apache.spark.memory.TestMemoryConsumer import org.apache.spark.util.NonSerializable // Common state shared by FailureSuite-launched tasks. We use a global object @@ -149,7 +150,8 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext { // cause is preserved val thrownDueToTaskFailure = intercept[SparkException] { sc.parallelize(Seq(0)).mapPartitions { iter => - TaskContext.get().taskMemoryManager().allocatePage(128, null) + val c = new TestMemoryConsumer(TaskContext.get().taskMemoryManager()) + TaskContext.get().taskMemoryManager().allocatePage(128, c) throw new Exception("intentional task failure") iter }.count() @@ -159,7 +161,8 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext { // If the task succeeded but memory was leaked, then the task should fail due to that leak val thrownDueToMemoryLeak = intercept[SparkException] { sc.parallelize(Seq(0)).mapPartitions { iter => - TaskContext.get().taskMemoryManager().allocatePage(128, null) + val c = new TestMemoryConsumer(TaskContext.get().taskMemoryManager()) + TaskContext.get().taskMemoryManager().allocatePage(128, c) iter }.count() } diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala index 688eb6bde904..840f55ce2f6e 100644 --- a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2} class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { @@ -213,7 +214,7 @@ class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { private class SaveAccumContextCleaner(sc: SparkContext) extends ContextCleaner(sc) { private val accumsRegistered = new ArrayBuffer[Long] - override def registerAccumulatorForCleanup(a: NewAccumulator[_, _]): Unit = { + override def registerAccumulatorForCleanup(a: AccumulatorV2[_, _]): Unit = { accumsRegistered += a.id super.registerAccumulatorForCleanup(a) } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index ddf48765ec30..c6aebc19fd12 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -22,6 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.mockito.Matchers.{any, isA} import org.mockito.Mockito._ +import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEnv} import org.apache.spark.scheduler.{CompressedMapStatus, MapStatus} import org.apache.spark.shuffle.FetchFailedException @@ -30,6 +31,12 @@ import org.apache.spark.storage.{BlockManagerId, ShuffleBlockId} class MapOutputTrackerSuite extends SparkFunSuite { private val conf = new SparkConf + private def newTrackerMaster(sparkConf: SparkConf = conf) = { + val broadcastManager = new BroadcastManager(true, sparkConf, + new SecurityManager(sparkConf)) + new MapOutputTrackerMaster(sparkConf, broadcastManager, true) + } + def createRpcEnv(name: String, host: String = "localhost", port: Int = 0, securityManager: SecurityManager = new SecurityManager(conf)): RpcEnv = { RpcEnv.create(name, host, port, conf, securityManager) @@ -37,7 +44,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { test("master start and stop") { val rpcEnv = createRpcEnv("test") - val tracker = new MapOutputTrackerMaster(conf) + val tracker = newTrackerMaster() tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.stop() @@ -46,7 +53,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { test("master register shuffle and fetch") { val rpcEnv = createRpcEnv("test") - val tracker = new MapOutputTrackerMaster(conf) + val tracker = newTrackerMaster() tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.registerShuffle(10, 2) @@ -62,13 +69,14 @@ class MapOutputTrackerSuite extends SparkFunSuite { Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))), (BlockManagerId("b", "hostB", 1000), ArrayBuffer((ShuffleBlockId(10, 1, 0), size10000)))) .toSet) + assert(0 == tracker.getNumCachedSerializedBroadcast) tracker.stop() rpcEnv.shutdown() } test("master register and unregister shuffle") { val rpcEnv = createRpcEnv("test") - val tracker = new MapOutputTrackerMaster(conf) + val tracker = newTrackerMaster() tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.registerShuffle(10, 2) @@ -80,6 +88,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { Array(compressedSize10000, compressedSize1000))) assert(tracker.containsShuffle(10)) assert(tracker.getMapSizesByExecutorId(10, 0).nonEmpty) + assert(0 == tracker.getNumCachedSerializedBroadcast) tracker.unregisterShuffle(10) assert(!tracker.containsShuffle(10)) assert(tracker.getMapSizesByExecutorId(10, 0).isEmpty) @@ -90,7 +99,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { test("master register shuffle and unregister map output and fetch") { val rpcEnv = createRpcEnv("test") - val tracker = new MapOutputTrackerMaster(conf) + val tracker = newTrackerMaster() tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.registerShuffle(10, 2) @@ -101,6 +110,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(compressedSize10000, compressedSize1000, compressedSize1000))) + assert(0 == tracker.getNumCachedSerializedBroadcast) // As if we had two simultaneous fetch failures tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) @@ -118,7 +128,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { val hostname = "localhost" val rpcEnv = createRpcEnv("spark", hostname, 0, new SecurityManager(conf)) - val masterTracker = new MapOutputTrackerMaster(conf) + val masterTracker = newTrackerMaster() masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) @@ -139,6 +149,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getMapSizesByExecutorId(10, 0) === Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))))) + assert(0 == masterTracker.getNumCachedSerializedBroadcast) masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) masterTracker.incrementEpoch() @@ -147,6 +158,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { // failure should be cached intercept[FetchFailedException] { slaveTracker.getMapSizesByExecutorId(10, 0) } + assert(0 == masterTracker.getNumCachedSerializedBroadcast) masterTracker.stop() slaveTracker.stop() @@ -158,8 +170,9 @@ class MapOutputTrackerSuite extends SparkFunSuite { val newConf = new SparkConf newConf.set("spark.rpc.message.maxSize", "1") newConf.set("spark.rpc.askTimeout", "1") // Fail fast + newConf.set("spark.shuffle.mapOutput.minSizeForBroadcast", "1048576") - val masterTracker = new MapOutputTrackerMaster(conf) + val masterTracker = newTrackerMaster(newConf) val rpcEnv = createRpcEnv("spark") val masterEndpoint = new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, newConf) rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) @@ -172,45 +185,27 @@ class MapOutputTrackerSuite extends SparkFunSuite { val rpcCallContext = mock(classOf[RpcCallContext]) when(rpcCallContext.senderAddress).thenReturn(senderAddress) masterEndpoint.receiveAndReply(rpcCallContext)(GetMapOutputStatuses(10)) - verify(rpcCallContext).reply(any()) - verify(rpcCallContext, never()).sendFailure(any()) + // Default size for broadcast in this testsuite is set to -1 so should not cause broadcast + // to be used. + verify(rpcCallContext, timeout(30000)).reply(any()) + assert(0 == masterTracker.getNumCachedSerializedBroadcast) // masterTracker.stop() // this throws an exception rpcEnv.shutdown() } - test("remote fetch exceeds max RPC message size") { + test("min broadcast size exceeds max RPC message size") { val newConf = new SparkConf newConf.set("spark.rpc.message.maxSize", "1") newConf.set("spark.rpc.askTimeout", "1") // Fail fast + newConf.set("spark.shuffle.mapOutput.minSizeForBroadcast", Int.MaxValue.toString) - val masterTracker = new MapOutputTrackerMaster(conf) - val rpcEnv = createRpcEnv("test") - val masterEndpoint = new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, newConf) - rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) - - // Message size should be ~1.1MB, and MapOutputTrackerMasterEndpoint should throw exception. - // Note that the size is hand-selected here because map output statuses are compressed before - // being sent. - masterTracker.registerShuffle(20, 100) - (0 until 100).foreach { i => - masterTracker.registerMapOutput(20, i, new CompressedMapStatus( - BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) - } - val senderAddress = RpcAddress("localhost", 12345) - val rpcCallContext = mock(classOf[RpcCallContext]) - when(rpcCallContext.senderAddress).thenReturn(senderAddress) - masterEndpoint.receiveAndReply(rpcCallContext)(GetMapOutputStatuses(20)) - verify(rpcCallContext, never()).reply(any()) - verify(rpcCallContext).sendFailure(isA(classOf[SparkException])) - -// masterTracker.stop() // this throws an exception - rpcEnv.shutdown() + intercept[IllegalArgumentException] { newTrackerMaster(newConf) } } test("getLocationsWithLargestOutputs with multiple outputs in same machine") { val rpcEnv = createRpcEnv("test") - val tracker = new MapOutputTrackerMaster(conf) + val tracker = newTrackerMaster() tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) // Setup 3 map tasks @@ -242,4 +237,44 @@ class MapOutputTrackerSuite extends SparkFunSuite { tracker.stop() rpcEnv.shutdown() } + + test("remote fetch using broadcast") { + val newConf = new SparkConf + newConf.set("spark.rpc.message.maxSize", "1") + newConf.set("spark.rpc.askTimeout", "1") // Fail fast + newConf.set("spark.shuffle.mapOutput.minSizeForBroadcast", "10240") // 10 KB << 1MB framesize + + // needs TorrentBroadcast so need a SparkContext + val sc = new SparkContext("local", "MapOutputTrackerSuite", newConf) + try { + val masterTracker = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + val rpcEnv = sc.env.rpcEnv + val masterEndpoint = new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, newConf) + rpcEnv.stop(masterTracker.trackerEndpoint) + rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) + + // Frame size should be ~1.1MB, and MapOutputTrackerMasterActor should throw exception. + // Note that the size is hand-selected here because map output statuses are compressed before + // being sent. + masterTracker.registerShuffle(20, 100) + (0 until 100).foreach { i => + masterTracker.registerMapOutput(20, i, new CompressedMapStatus( + BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) + } + val senderAddress = RpcAddress("localhost", 12345) + val rpcCallContext = mock(classOf[RpcCallContext]) + when(rpcCallContext.senderAddress).thenReturn(senderAddress) + masterEndpoint.receiveAndReply(rpcCallContext)(GetMapOutputStatuses(20)) + // should succeed since majority of data is broadcast and actual serialized + // message size is small + verify(rpcCallContext, timeout(30000)).reply(any()) + assert(1 == masterTracker.getNumCachedSerializedBroadcast) + masterTracker.unregisterShuffle(20) + assert(0 == masterTracker.getNumCachedSerializedBroadcast) + + } finally { + LocalSparkContext.stop(sc) + } + } + } diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index 8bdb237c28f6..9801b2638cc1 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -19,8 +19,18 @@ package org.apache.spark import java.io.File +import org.apache.spark.security.GroupMappingServiceProvider import org.apache.spark.util.{ResetSystemProperties, SparkConfWithEnv, Utils} +class DummyGroupMappingServiceProvider extends GroupMappingServiceProvider { + + val userGroups: Set[String] = Set[String]("group1", "group2", "group3") + + override def getGroups(username: String): Set[String] = { + userGroups + } +} + class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with conf") { @@ -37,6 +47,45 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions("user3") === false) } + test("set security with conf for groups") { + val conf = new SparkConf + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + conf.set("spark.ui.acls.enable", "true") + conf.set("spark.ui.view.acls.groups", "group1,group2") + // default ShellBasedGroupsMappingProvider is used to resolve user groups + val securityManager = new SecurityManager(conf); + // assuming executing user does not belong to group1,group2 + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + + val conf2 = new SparkConf + conf2.set("spark.authenticate", "true") + conf2.set("spark.authenticate.secret", "good") + conf2.set("spark.ui.acls.enable", "true") + conf2.set("spark.ui.view.acls.groups", "group1,group2") + // explicitly specify a custom GroupsMappingServiceProvider + conf2.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager2 = new SecurityManager(conf2); + // group4,group5 do not match + assert(securityManager2.checkUIViewPermissions("user1") === true) + assert(securityManager2.checkUIViewPermissions("user2") === true) + + val conf3 = new SparkConf + conf3.set("spark.authenticate", "true") + conf3.set("spark.authenticate.secret", "good") + conf3.set("spark.ui.acls.enable", "true") + conf3.set("spark.ui.view.acls.groups", "group4,group5") + // explicitly specify a bogus GroupsMappingServiceProvider + conf3.set("spark.user.groups.mapping", "BogusServiceProvider") + + val securityManager3 = new SecurityManager(conf3); + // BogusServiceProvider cannot be loaded and an error is logged returning an empty group set + assert(securityManager3.checkUIViewPermissions("user1") === false) + assert(securityManager3.checkUIViewPermissions("user2") === false) + } + test("set security with api") { val conf = new SparkConf conf.set("spark.ui.view.acls", "user1,user2") @@ -60,6 +109,40 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions(null) === true) } + test("set security with api for groups") { + val conf = new SparkConf + conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + securityManager.setViewAclsGroups("group1,group2") + + // group1,group2 match + assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user2") === true) + + // change groups so they do not match + securityManager.setViewAclsGroups("group4,group5") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + + val conf2 = new SparkConf + conf.set("spark.user.groups.mapping", "BogusServiceProvider") + + val securityManager2 = new SecurityManager(conf2) + securityManager2.setAcls(true) + securityManager2.setViewAclsGroups("group1,group2") + + // group1,group2 do not match because of BogusServiceProvider + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + + // setting viewAclsGroups to empty should still not match because of BogusServiceProvider + securityManager2.setViewAclsGroups("") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + } + test("set security modify acls") { val conf = new SparkConf conf.set("spark.modify.acls", "user1,user2") @@ -84,6 +167,29 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkModifyPermissions(null) === true) } + test("set security modify acls for groups") { + val conf = new SparkConf + conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + securityManager.setModifyAclsGroups("group1,group2") + + // group1,group2 match + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkModifyPermissions("user2") === true) + + // change groups so they do not match + securityManager.setModifyAclsGroups("group4,group5") + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user2") === false) + + // change so they match again + securityManager.setModifyAclsGroups("group2,group3") + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkModifyPermissions("user2") === true) + } + test("set security admin acls") { val conf = new SparkConf conf.set("spark.admin.acls", "user1,user2") @@ -122,7 +228,48 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user3") === false) assert(securityManager.checkUIViewPermissions(null) === true) + } + + test("set security admin acls for groups") { + val conf = new SparkConf + conf.set("spark.admin.acls.groups", "group1") + conf.set("spark.ui.view.acls.groups", "group2") + conf.set("spark.modify.acls.groups", "group3") + conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + assert(securityManager.aclsEnabled() === true) + + // group1,group2,group3 match + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user1") === true) + // change admin groups so they do not match. view and modify groups are set to admin groups + securityManager.setAdminAclsGroups("group4,group5") + // invoke the set ui and modify to propagate the changes + securityManager.setViewAclsGroups("") + securityManager.setModifyAclsGroups("") + + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user1") === false) + + // change modify groups so they match + securityManager.setModifyAclsGroups("group3") + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user1") === false) + + // change view groups so they match + securityManager.setViewAclsGroups("group2") + securityManager.setModifyAclsGroups("group4") + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user1") === true) + + // change modify and view groups so they do not match + securityManager.setViewAclsGroups("group7") + securityManager.setModifyAclsGroups("group8") + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user1") === false) } test("set security with * in acls") { @@ -166,6 +313,57 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkModifyPermissions("user8") === true) } + test("set security with * in acls for groups") { + val conf = new SparkConf + conf.set("spark.ui.acls.enable", "true") + conf.set("spark.admin.acls.groups", "group4,group5") + conf.set("spark.ui.view.acls.groups", "*") + conf.set("spark.modify.acls.groups", "group6") + + val securityManager = new SecurityManager(conf) + assert(securityManager.aclsEnabled() === true) + + // check for viewAclsGroups with * + assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user2") === true) + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user2") === false) + + // check for modifyAcls with * + securityManager.setModifyAclsGroups("*") + securityManager.setViewAclsGroups("group6") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkModifyPermissions("user2") === true) + + // check for adminAcls with * + securityManager.setAdminAclsGroups("group9,*") + securityManager.setModifyAclsGroups("group4,group5") + securityManager.setViewAclsGroups("group6,group7") + assert(securityManager.checkUIViewPermissions("user5") === true) + assert(securityManager.checkUIViewPermissions("user6") === true) + assert(securityManager.checkModifyPermissions("user7") === true) + assert(securityManager.checkModifyPermissions("user8") === true) + } + + test("security for groups default behavior") { + // no groups or userToGroupsMapper provided + // this will default to the ShellBasedGroupsMappingProvider + val conf = new SparkConf + + val securityManager = new SecurityManager(conf) + securityManager.setAcls(true) + + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user1") === false) + + // set groups only + securityManager.setAdminAclsGroups("group1,group2") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user1") === false) + } + test("ssl on setup") { val conf = SSLSampleConfigs.sparkSSLConfig() val expectedAlgorithms = Set( diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index a759f364fe05..63987084ffc0 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -323,4 +323,32 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext { assert(sc.getConf.getInt("spark.executor.instances", 0) === 6) } } + + + test("localProperties are inherited by spawned threads.") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.setLocalProperty("testProperty", "testValue") + var result = "unset"; + val thread = new Thread() { override def run() = {result = sc.getLocalProperty("testProperty")}} + thread.start() + thread.join() + sc.stop() + assert(result == "testValue") + } + + test("localProperties do not cross-talk between threads.") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + var result = "unset"; + val thread1 = new Thread() { + override def run() = {sc.setLocalProperty("testProperty", "testValue")}} + // testProperty should be unset and thus return null + val thread2 = new Thread() { + override def run() = {result = sc.getLocalProperty("testProperty")}} + thread1.start() + thread1.join() + thread2.start() + thread2.join() + sc.stop() + assert(result == null) + } } diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 4aae2c9b4a8e..0081bca63959 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -21,6 +21,7 @@ package org.apache.spark import org.scalatest.{BeforeAndAfterAll, FunSuite, Outcome} import org.apache.spark.internal.Logging +import org.apache.spark.util.AccumulatorContext /** * Base abstract class for all unit tests in Spark for handling common functionality. diff --git a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala index 0c9382a92bca..69a460fbc7db 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala @@ -17,74 +17,96 @@ package org.apache.spark.deploy.master.ui +import java.io.DataOutputStream +import java.net.{HttpURLConnection, URL} +import java.nio.charset.StandardCharsets import java.util.Date -import scala.io.Source -import scala.language.postfixOps +import scala.collection.mutable.HashMap -import org.json4s.jackson.JsonMethods._ -import org.json4s.JsonAST.{JInt, JNothing, JString} -import org.mockito.Mockito.{mock, when} -import org.scalatest.BeforeAndAfter +import org.mockito.Mockito.{mock, times, verify, when} +import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -import org.apache.spark.deploy.DeployMessages.MasterStateResponse +import org.apache.spark.deploy.DeployMessages.{KillDriverResponse, RequestKillDriver} import org.apache.spark.deploy.DeployTestUtils._ import org.apache.spark.deploy.master._ -import org.apache.spark.rpc.RpcEnv +import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv} -class MasterWebUISuite extends SparkFunSuite with BeforeAndAfter { +class MasterWebUISuite extends SparkFunSuite with BeforeAndAfterAll { - val masterPage = mock(classOf[MasterPage]) - val master = { - val conf = new SparkConf - val securityMgr = new SecurityManager(conf) - val rpcEnv = RpcEnv.create(Master.SYSTEM_NAME, "localhost", 0, conf, securityMgr) - val master = new Master(rpcEnv, rpcEnv.address, 0, securityMgr, conf) - master - } - val masterWebUI = new MasterWebUI(master, 0, customMasterPage = Some(masterPage)) + val conf = new SparkConf + val securityMgr = new SecurityManager(conf) + val rpcEnv = mock(classOf[RpcEnv]) + val master = mock(classOf[Master]) + val masterEndpointRef = mock(classOf[RpcEndpointRef]) + when(master.securityMgr).thenReturn(securityMgr) + when(master.conf).thenReturn(conf) + when(master.rpcEnv).thenReturn(rpcEnv) + when(master.self).thenReturn(masterEndpointRef) + val masterWebUI = new MasterWebUI(master, 0) - before { + override def beforeAll() { + super.beforeAll() masterWebUI.bind() } - after { + override def afterAll() { masterWebUI.stop() + super.afterAll() } - test("list applications") { - val worker = createWorkerInfo() + test("kill application") { val appDesc = createAppDesc() // use new start date so it isn't filtered by UI val activeApp = new ApplicationInfo( - new Date().getTime, "id", appDesc, new Date(), null, Int.MaxValue) - activeApp.addExecutor(worker, 2) - - val workers = Array[WorkerInfo](worker) - val activeApps = Array(activeApp) - val completedApps = Array[ApplicationInfo]() - val activeDrivers = Array[DriverInfo]() - val completedDrivers = Array[DriverInfo]() - val stateResponse = new MasterStateResponse( - "host", 8080, None, workers, activeApps, completedApps, - activeDrivers, completedDrivers, RecoveryState.ALIVE) - - when(masterPage.getMasterState).thenReturn(stateResponse) - - val resultJson = Source.fromURL( - s"http://localhost:${masterWebUI.boundPort}/api/v1/applications") - .mkString - val parsedJson = parse(resultJson) - val firstApp = parsedJson(0) - - assert(firstApp \ "id" === JString(activeApp.id)) - assert(firstApp \ "name" === JString(activeApp.desc.name)) - assert(firstApp \ "coresGranted" === JInt(2)) - assert(firstApp \ "maxCores" === JInt(4)) - assert(firstApp \ "memoryPerExecutorMB" === JInt(1234)) - assert(firstApp \ "coresPerExecutor" === JNothing) + new Date().getTime, "app-0", appDesc, new Date(), null, Int.MaxValue) + + when(master.idToApp).thenReturn(HashMap[String, ApplicationInfo]((activeApp.id, activeApp))) + + val url = s"http://localhost:${masterWebUI.boundPort}/app/kill/" + val body = convPostDataToString(Map(("id", activeApp.id), ("terminate", "true"))) + val conn = sendHttpRequest(url, "POST", body) + conn.getResponseCode + + // Verify the master was called to remove the active app + verify(master, times(1)).removeApplication(activeApp, ApplicationState.KILLED) + } + + test("kill driver") { + val activeDriverId = "driver-0" + val url = s"http://localhost:${masterWebUI.boundPort}/driver/kill/" + val body = convPostDataToString(Map(("id", activeDriverId), ("terminate", "true"))) + val conn = sendHttpRequest(url, "POST", body) + conn.getResponseCode + + // Verify that master was asked to kill driver with the correct id + verify(masterEndpointRef, times(1)).ask[KillDriverResponse](RequestKillDriver(activeDriverId)) } + private def convPostDataToString(data: Map[String, String]): String = { + (for ((name, value) <- data) yield s"$name=$value").mkString("&") + } + + /** + * Send an HTTP request to the given URL using the method and the body specified. + * Return the connection object. + */ + private def sendHttpRequest( + url: String, + method: String, + body: String = ""): HttpURLConnection = { + val conn = new URL(url).openConnection().asInstanceOf[HttpURLConnection] + conn.setRequestMethod(method) + if (body.nonEmpty) { + conn.setDoOutput(true) + conn.setRequestProperty("Content-Type", "application/x-www-form-urlencoded") + conn.setRequestProperty("Content-Length", Integer.toString(body.length)) + val out = new DataOutputStream(conn.getOutputStream) + out.write(body.getBytes(StandardCharsets.UTF_8)) + out.close() + } + conn + } } diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index 94f6e1a3a77c..eae26fa742a2 100644 --- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.Assertions import org.apache.spark._ import org.apache.spark.storage.{BlockStatus, StorageLevel, TestBlockId} +import org.apache.spark.util.AccumulatorV2 class TaskMetricsSuite extends SparkFunSuite { @@ -203,7 +204,7 @@ class TaskMetricsSuite extends SparkFunSuite { acc1.add(1) acc2.add(2) val newUpdates = tm.accumulators() - .map(a => (a.id, a.asInstanceOf[NewAccumulator[Any, Any]])).toMap + .map(a => (a.id, a.asInstanceOf[AccumulatorV2[Any, Any]])).toMap assert(newUpdates.contains(acc1.id)) assert(newUpdates.contains(acc2.id)) assert(newUpdates.contains(acc3.id)) @@ -230,8 +231,8 @@ private[spark] object TaskMetricsSuite extends Assertions { * Note: this does NOT check accumulator ID equality. */ def assertUpdatesEquals( - updates1: Seq[NewAccumulator[_, _]], - updates2: Seq[NewAccumulator[_, _]]): Unit = { + updates1: Seq[AccumulatorV2[_, _]], + updates2: Seq[AccumulatorV2[_, _]]): Unit = { assert(updates1.size === updates2.size) updates1.zip(updates2).foreach { case (acc1, acc2) => // do not assert ID equals here diff --git a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala index 713560d3ddfa..cac15a1dc441 100644 --- a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala @@ -48,7 +48,7 @@ class LauncherBackendSuite extends SparkFunSuite with Matchers { .setConf("spark.ui.enabled", "false") .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, s"-Dtest.appender=console") .setMaster(master) - .setAppResource("spark-internal") + .setAppResource(SparkLauncher.NO_RESOURCE) .setMainClass(TestApp.getClass.getName().stripSuffix("$")) .startApplication() diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index a1286523a235..38bf7e5e5aec 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -78,6 +78,21 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft ms } + /** + * Make a mocked [[MemoryStore]] whose [[MemoryStore.evictBlocksToFreeSpace]] method is + * stubbed to always throw [[RuntimeException]]. + */ + protected def makeBadMemoryStore(mm: MemoryManager): MemoryStore = { + val ms = mock(classOf[MemoryStore], RETURNS_SMART_NULLS) + when(ms.evictBlocksToFreeSpace(any(), anyLong(), any())).thenAnswer(new Answer[Long] { + override def answer(invocation: InvocationOnMock): Long = { + throw new RuntimeException("bad memory store!") + } + }) + mm.setMemoryStore(ms) + ms + } + /** * Simulate the part of [[MemoryStore.evictBlocksToFreeSpace]] that releases storage memory. * @@ -147,39 +162,42 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft test("single task requesting on-heap execution memory") { val manager = createMemoryManager(1000L) val taskMemoryManager = new TaskMemoryManager(manager, 0) + val c = new TestMemoryConsumer(taskMemoryManager) - assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 100L) - assert(taskMemoryManager.acquireExecutionMemory(400L, MemoryMode.ON_HEAP, null) === 400L) - assert(taskMemoryManager.acquireExecutionMemory(400L, MemoryMode.ON_HEAP, null) === 400L) - assert(taskMemoryManager.acquireExecutionMemory(200L, MemoryMode.ON_HEAP, null) === 100L) - assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 0L) - assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 0L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 100L) + assert(taskMemoryManager.acquireExecutionMemory(400L, c) === 400L) + assert(taskMemoryManager.acquireExecutionMemory(400L, c) === 400L) + assert(taskMemoryManager.acquireExecutionMemory(200L, c) === 100L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 0L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 0L) - taskMemoryManager.releaseExecutionMemory(500L, MemoryMode.ON_HEAP, null) - assert(taskMemoryManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) === 300L) - assert(taskMemoryManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) === 200L) + taskMemoryManager.releaseExecutionMemory(500L, c) + assert(taskMemoryManager.acquireExecutionMemory(300L, c) === 300L) + assert(taskMemoryManager.acquireExecutionMemory(300L, c) === 200L) taskMemoryManager.cleanUpAllAllocatedMemory() - assert(taskMemoryManager.acquireExecutionMemory(1000L, MemoryMode.ON_HEAP, null) === 1000L) - assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 0L) + assert(taskMemoryManager.acquireExecutionMemory(1000L, c) === 1000L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 0L) } test("two tasks requesting full on-heap execution memory") { val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // Have both tasks request 500 bytes, then wait until both requests have been granted: - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(500L, c1) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 500L) assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 500L) // Have both tasks each request 500 bytes more; both should immediately return 0 as they are // both now at 1 / N - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, c1) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } assert(ThreadUtils.awaitResult(t1Result2, 200.millis) === 0L) assert(ThreadUtils.awaitResult(t2Result2, 200.millis) === 0L) } @@ -188,18 +206,20 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // Have both tasks request 250 bytes, then wait until both requests have been granted: - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(250L, MemoryMode.ON_HEAP, null) } - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, MemoryMode.ON_HEAP, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(250L, c1) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, c2) } assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 250L) assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 250L) // Have both tasks each request 500 bytes more. // We should only grant 250 bytes to each of them on this second request - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, c1) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } assert(ThreadUtils.awaitResult(t1Result2, futureTimeout) === 250L) assert(ThreadUtils.awaitResult(t2Result2, futureTimeout) === 250L) } @@ -208,20 +228,22 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, MemoryMode.ON_HEAP, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, c1) } assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 1000L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, MemoryMode.ON_HEAP, null) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, c2) } // Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult // to make sure the other thread blocks for some time otherwise. Thread.sleep(300) - t1MemManager.releaseExecutionMemory(250L, MemoryMode.ON_HEAP, null) + t1MemManager.releaseExecutionMemory(250L, c1) // The memory freed from t1 should now be granted to t2. assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 250L) // Further requests by t2 should be denied immediately because it now has 1 / 2N of the memory. - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(100L, c2) } assert(ThreadUtils.awaitResult(t2Result2, 200.millis) === 0L) } @@ -229,21 +251,23 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, MemoryMode.ON_HEAP, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, c1) } assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 1000L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } // Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult // to make sure the other thread blocks for some time otherwise. Thread.sleep(300) // t1 releases all of its memory, so t2 should be able to grab all of the memory t1MemManager.cleanUpAllAllocatedMemory() assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 500L) - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } assert(ThreadUtils.awaitResult(t2Result2, futureTimeout) === 500L) - val t2Result3 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t2Result3 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } assert(ThreadUtils.awaitResult(t2Result3, 200.millis) === 0L) } @@ -252,15 +276,17 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(700L, MemoryMode.ON_HEAP, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(700L, c1) } assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 700L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(300L, c2) } assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 300L) - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) } + val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L, c1) } assert(ThreadUtils.awaitResult(t1Result2, 200.millis) === 0L) } @@ -270,17 +296,18 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft maxOffHeapExecutionMemory = 1000L) val tMemManager = new TaskMemoryManager(memoryManager, 1) - val result1 = Future { tMemManager.acquireExecutionMemory(1000L, MemoryMode.OFF_HEAP, null) } + val c = new TestMemoryConsumer(tMemManager, MemoryMode.OFF_HEAP) + val result1 = Future { tMemManager.acquireExecutionMemory(1000L, c) } assert(ThreadUtils.awaitResult(result1, 200.millis) === 1000L) assert(tMemManager.getMemoryConsumptionForThisTask === 1000L) - val result2 = Future { tMemManager.acquireExecutionMemory(300L, MemoryMode.OFF_HEAP, null) } + val result2 = Future { tMemManager.acquireExecutionMemory(300L, c) } assert(ThreadUtils.awaitResult(result2, 200.millis) === 0L) assert(tMemManager.getMemoryConsumptionForThisTask === 1000L) - tMemManager.releaseExecutionMemory(500L, MemoryMode.OFF_HEAP, null) + tMemManager.releaseExecutionMemory(500L, c) assert(tMemManager.getMemoryConsumptionForThisTask === 500L) - tMemManager.releaseExecutionMemory(500L, MemoryMode.OFF_HEAP, null) + tMemManager.releaseExecutionMemory(500L, c) assert(tMemManager.getMemoryConsumptionForThisTask === 0L) } } diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index 14255818c7b5..c821054412d7 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -280,4 +280,27 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes assert(evictedBlocks.nonEmpty) } + test("SPARK-15260: atomically resize memory pools") { + val conf = new SparkConf() + .set("spark.memory.fraction", "1") + .set("spark.memory.storageFraction", "0") + .set("spark.testing.memory", "1000") + val mm = UnifiedMemoryManager(conf, numCores = 2) + makeBadMemoryStore(mm) + val memoryMode = MemoryMode.ON_HEAP + // Acquire 1000 then release 600 bytes of storage memory, leaving the + // storage memory pool at 1000 bytes but only 400 bytes of which are used. + assert(mm.acquireStorageMemory(dummyBlock, 1000L, memoryMode)) + mm.releaseStorageMemory(600L, memoryMode) + // Before the fix for SPARK-15260, we would first shrink the storage pool by the amount of + // unused storage memory (600 bytes), try to evict blocks, then enlarge the execution pool + // by the same amount. If the eviction threw an exception, then we would shrink one pool + // without enlarging the other, resulting in an assertion failure. + intercept[RuntimeException] { + mm.acquireExecutionMemory(1000L, 0, memoryMode) + } + val assertInvariants = PrivateMethod[Unit]('assertInvariants) + mm.invokePrivate[Unit](assertInvariants()) + } + } diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index 8cb0a295b077..58664e77d24a 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -65,9 +65,9 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim test("foreachAsync") { zeroPartRdd.foreachAsync(i => Unit).get() - val accum = sc.accumulator(0) + val accum = sc.longAccumulator sc.parallelize(1 to 1000, 3).foreachAsync { i => - accum += 1 + accum.add(1) }.get() assert(accum.value === 1000) } @@ -75,9 +75,9 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim test("foreachPartitionAsync") { zeroPartRdd.foreachPartitionAsync(iter => Unit).get() - val accum = sc.accumulator(0) + val accum = sc.longAccumulator sc.parallelize(1 to 1000, 9).foreachPartitionAsync { iter => - accum += 1 + accum.add(1) }.get() assert(accum.value === 9) } diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index e9cc8195240f..fe2058d61389 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -171,7 +171,7 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext { val pipedPwd = nums.pipe(Seq("pwd"), separateWorkingDir = true) val collectPwd = pipedPwd.collect() assert(collectPwd(0).contains("tasks/")) - val pipedLs = nums.pipe(Seq("ls"), separateWorkingDir = true).collect() + val pipedLs = nums.pipe(Seq("ls"), separateWorkingDir = true, bufferSize = 16384).collect() // make sure symlinks were created assert(pipedLs.length > 0) // clean up top level tasks directory diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 8dc463d56d18..979fb426c948 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -116,6 +116,23 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assert(sc.union(Seq(nums, nums)).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) } + test("SparkContext.union parallel partition listing") { + val nums1 = sc.makeRDD(Array(1, 2, 3, 4), 2) + val nums2 = sc.makeRDD(Array(5, 6, 7, 8), 2) + val serialUnion = sc.union(nums1, nums2) + val expected = serialUnion.collect().toList + + assert(serialUnion.asInstanceOf[UnionRDD[Int]].isPartitionListingParallel === false) + + sc.conf.set("spark.rdd.parallelListingThreshold", "1") + val parallelUnion = sc.union(nums1, nums2) + val actual = parallelUnion.collect().toList + sc.conf.remove("spark.rdd.parallelListingThreshold") + + assert(parallelUnion.asInstanceOf[UnionRDD[Int]].isPartitionListingParallel === true) + assert(expected === actual) + } + test("SparkContext.union creates UnionRDD if at least one RDD has no partitioner") { val rddWithPartitioner = sc.parallelize(Seq(1 -> true)).partitionBy(new HashPartitioner(1)) val rddWithNoPartitioner = sc.parallelize(Seq(2 -> true)) @@ -377,6 +394,33 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { map{x => List(x)}.toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back") } + test("coalesced RDDs with partial locality") { + // Make an RDD that has some locality preferences and some without. This can happen + // with UnionRDD + val data = sc.makeRDD((1 to 9).map(i => { + if (i > 4) { + (i, (i to (i + 2)).map { j => "m" + (j % 6) }) + } else { + (i, Vector()) + } + })) + val coalesced1 = data.coalesce(3) + assert(coalesced1.collect().toList.sorted === (1 to 9).toList, "Data got *lost* in coalescing") + + val splits = coalesced1.glom().collect().map(_.toList).toList + assert(splits.length === 3, "Supposed to coalesce to 3 but got " + splits.length) + + assert(splits.forall(_.length >= 1) === true, "Some partitions were empty") + + // If we try to coalesce into more partitions than the original RDD, it should just + // keep the original number of partitions. + val coalesced4 = data.coalesce(20) + val listOfLists = coalesced4.glom().collect().map(_.toList).toList + val sortedList = listOfLists.sortWith{ (x, y) => !x.isEmpty && (y.isEmpty || (x(0) < y(0))) } + assert(sortedList === (1 to 9). + map{x => List(x)}.toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back") + } + test("coalesced RDDs with locality, large scale (10K partitions)") { // large scale experiment import collection.mutable @@ -418,6 +462,48 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { } } + test("coalesced RDDs with partial locality, large scale (10K partitions)") { + // large scale experiment + import collection.mutable + val halfpartitions = 5000 + val partitions = 10000 + val numMachines = 50 + val machines = mutable.ListBuffer[String]() + (1 to numMachines).foreach(machines += "m" + _) + val rnd = scala.util.Random + for (seed <- 1 to 5) { + rnd.setSeed(seed) + + val firstBlocks = (1 to halfpartitions).map { i => + (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) + } + val blocksNoLocality = (halfpartitions + 1 to partitions).map { i => + (i, List()) + } + val blocks = firstBlocks ++ blocksNoLocality + + val data2 = sc.makeRDD(blocks) + + // first try going to same number of partitions + val coalesced2 = data2.coalesce(partitions) + + // test that we have 10000 partitions + assert(coalesced2.partitions.size == 10000, "Expected 10000 partitions, but got " + + coalesced2.partitions.size) + + // test that we have 100 partitions + val coalesced3 = data2.coalesce(numMachines * 2) + assert(coalesced3.partitions.size == 100, "Expected 100 partitions, but got " + + coalesced3.partitions.size) + + // test that the groups are load balanced with 100 +/- 20 elements in each + val maxImbalance3 = coalesced3.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size) + .foldLeft(0)((dev, curr) => math.max(math.abs(100 - curr), dev)) + assert(maxImbalance3 <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance3) + } + } + // Test for SPARK-2412 -- ensure that the second pass of the algorithm does not throw an exception test("coalesced RDDs with locality, fail first pass") { val initialPartitions = 1000 diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 9912d1f3bc5a..088a47608621 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -28,11 +28,11 @@ import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} -import org.apache.spark.util.{CallSite, Utils} +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, LongAccumulator, Utils} class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) extends DAGSchedulerEventProcessLoop(dagScheduler) { @@ -112,7 +112,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou override def stop() = {} override def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean = true override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager @@ -157,6 +157,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou } var mapOutputTracker: MapOutputTrackerMaster = null + var broadcastManager: BroadcastManager = null + var securityMgr: SecurityManager = null var scheduler: DAGScheduler = null var dagEventProcessLoopTester: DAGSchedulerEventProcessLoop = null @@ -208,7 +210,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou cancelledStages.clear() cacheLocations.clear() results.clear() - mapOutputTracker = new MapOutputTrackerMaster(conf) + securityMgr = new SecurityManager(conf) + broadcastManager = new BroadcastManager(true, conf, securityMgr) + mapOutputTracker = new MapOutputTrackerMaster(conf, broadcastManager, true) scheduler = new DAGScheduler( sc, taskScheduler, @@ -321,6 +325,53 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou assert(sparkListener.stageByOrderOfExecution(0) < sparkListener.stageByOrderOfExecution(1)) } + /** + * This test ensures that DAGScheduler build stage graph correctly. + * + * Suppose you have the following DAG: + * + * [A] <--(s_A)-- [B] <--(s_B)-- [C] <--(s_C)-- [D] + * \ / + * <------------- + * + * Here, RDD B has a shuffle dependency on RDD A, and RDD C has shuffle dependency on both + * B and A. The shuffle dependency IDs are numbers in the DAGScheduler, but to make the example + * easier to understand, let's call the shuffled data from A shuffle dependency ID s_A and the + * shuffled data from B shuffle dependency ID s_B. + * + * Note: [] means an RDD, () means a shuffle dependency. + */ + test("[SPARK-13902] Ensure no duplicate stages are created") { + val rddA = new MyRDD(sc, 1, Nil) + val shuffleDepA = new ShuffleDependency(rddA, new HashPartitioner(1)) + val s_A = shuffleDepA.shuffleId + + val rddB = new MyRDD(sc, 1, List(shuffleDepA), tracker = mapOutputTracker) + val shuffleDepB = new ShuffleDependency(rddB, new HashPartitioner(1)) + val s_B = shuffleDepB.shuffleId + + val rddC = new MyRDD(sc, 1, List(shuffleDepA, shuffleDepB), tracker = mapOutputTracker) + val shuffleDepC = new ShuffleDependency(rddC, new HashPartitioner(1)) + val s_C = shuffleDepC.shuffleId + + val rddD = new MyRDD(sc, 1, List(shuffleDepC), tracker = mapOutputTracker) + + submit(rddD, Array(0)) + + assert(scheduler.shuffleToMapStage.size === 3) + assert(scheduler.activeJobs.size === 1) + + val mapStageA = scheduler.shuffleToMapStage(s_A) + val mapStageB = scheduler.shuffleToMapStage(s_B) + val mapStageC = scheduler.shuffleToMapStage(s_C) + val finalStage = scheduler.activeJobs.head.finalStage + + assert(mapStageA.parents.isEmpty) + assert(mapStageB.parents === List(mapStageA)) + assert(mapStageC.parents === List(mapStageA, mapStageB)) + assert(finalStage.parents === List(mapStageC)) + } + test("zero split job") { var numResults = 0 var failureReason: Option[Exception] = None @@ -483,7 +534,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou override def defaultParallelism(): Int = 2 override def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean = true override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def applicationAttemptId(): Option[String] = None @@ -2012,7 +2063,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou task: Task[_], reason: TaskEndReason, result: Any, - extraAccumUpdates: Seq[NewAccumulator[_, _]] = Seq.empty, + extraAccumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty, taskInfo: TaskInfo = createFakeTaskInfo()): CompletionEvent = { val accumUpdates = reason match { case Success => task.metrics.accumulators() diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index 16027d944fdf..59c1b359a780 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -17,9 +17,10 @@ package org.apache.spark.scheduler -import org.apache.spark.{LocalSparkContext, NewAccumulator, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.AccumulatorV2 class ExternalClusterManagerSuite extends SparkFunSuite with LocalSparkContext { test("launch of backend and scheduler") { @@ -67,6 +68,6 @@ private class DummyTaskScheduler extends TaskScheduler { override def applicationAttemptId(): Option[String] = None def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean = true } diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala index 601f1c378c41..32cdf16dd331 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala @@ -37,7 +37,6 @@ class OutputCommitCoordinatorIntegrationSuite override def beforeAll(): Unit = { super.beforeAll() val conf = new SparkConf() - .set("master", "local[2,4]") .set("spark.hadoop.outputCommitCoordination.enabled", "true") .set("spark.hadoop.mapred.output.committer.class", classOf[ThrowExceptionOnFirstAttemptOutputCommitter].getCanonicalName) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 9aca4dbc2364..368668bc7e2e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -168,8 +168,10 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark test("failed tasks collect only accumulators whose values count during failures") { sc = new SparkContext("local", "test") - val acc1 = AccumulatorSuite.createLongAccum("x", true) - val acc2 = AccumulatorSuite.createLongAccum("y", false) + val acc1 = AccumulatorSuite.createLongAccum("x", false) + val acc2 = AccumulatorSuite.createLongAccum("y", true) + acc1.add(1) + acc2.add(1) // Create a dummy task. We won't end up running this; we just want to collect // accumulator updates from it. val taskMetrics = TaskMetrics.empty @@ -185,12 +187,33 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark } // First, simulate task success. This should give us all the accumulators. val accumUpdates1 = task.collectAccumulatorUpdates(taskFailed = false) - val accumUpdates2 = taskMetrics.internalAccums ++ Seq(acc1, acc2) - TaskMetricsSuite.assertUpdatesEquals(accumUpdates1, accumUpdates2) + TaskMetricsSuite.assertUpdatesEquals(accumUpdates1.takeRight(2), Seq(acc1, acc2)) // Now, simulate task failures. This should give us only the accums that count failed values. - val accumUpdates3 = task.collectAccumulatorUpdates(taskFailed = true) - val accumUpdates4 = taskMetrics.internalAccums ++ Seq(acc1) - TaskMetricsSuite.assertUpdatesEquals(accumUpdates3, accumUpdates4) + val accumUpdates2 = task.collectAccumulatorUpdates(taskFailed = true) + TaskMetricsSuite.assertUpdatesEquals(accumUpdates2.takeRight(1), Seq(acc2)) + } + + test("only updated internal accumulators will be sent back to driver") { + sc = new SparkContext("local", "test") + // Create a dummy task. We won't end up running this; we just want to collect + // accumulator updates from it. + val taskMetrics = TaskMetrics.empty + val task = new Task[Int](0, 0, 0) { + context = new TaskContextImpl(0, 0, 0L, 0, + new TaskMemoryManager(SparkEnv.get.memoryManager, 0L), + new Properties, + SparkEnv.get.metricsSystem, + taskMetrics) + taskMetrics.incMemoryBytesSpilled(10) + override def runTask(tc: TaskContext): Int = 0 + } + val updatedAccums = task.collectAccumulatorUpdates() + assert(updatedAccums.length == 2) + // the RESULT_SIZE accumulator will be sent back anyway. + assert(updatedAccums(0).name == Some(InternalAccumulator.RESULT_SIZE)) + assert(updatedAccums(0).value == 0) + assert(updatedAccums(1).name == Some(InternalAccumulator.MEMORY_BYTES_SPILLED)) + assert(updatedAccums(1).value == 10) } test("localProperties are propagated to executors correctly") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 339fc4254d53..9b7b945bf367 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark._ import org.apache.spark.internal.Logging -import org.apache.spark.util.ManualClock +import org.apache.spark.util.{AccumulatorV2, ManualClock} class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) extends DAGScheduler(sc) { @@ -37,7 +37,7 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Seq[NewAccumulator[_, _]], + accumUpdates: Seq[AccumulatorV2[_, _]], taskInfo: TaskInfo) { taskScheduler.endedTasks(taskInfo.index) = reason } @@ -184,7 +184,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(3) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) - val accumUpdatesByTask: Array[Seq[NewAccumulator[_, _]]] = taskSet.tasks.map { task => + val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => task.metrics.internalAccums } @@ -791,7 +791,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg private def createTaskResult( id: Int, - accumUpdates: Seq[NewAccumulator[_, _]] = Seq.empty): DirectTaskResult[Int] = { + accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty): DirectTaskResult[Int] = { val valueSer = SparkEnv.get.serializer.newInstance() new DirectTaskResult[Int](valueSer.serialize(id), accumUpdates) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala index b18f0eb162b1..15d59e705291 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala @@ -147,6 +147,19 @@ class CoarseMesosSchedulerBackendSuite extends SparkFunSuite verifyDeclinedOffer(driver, createOfferId("o1"), true) } + test("mesos declines offers with a filter when reached spark.cores.max") { + val maxCores = 3 + setBackend(Map("spark.cores.max" -> maxCores.toString)) + + val executorMemory = backend.executorMemory(sc) + offerResources(List( + (executorMemory, maxCores + 1), + (executorMemory, maxCores + 1))) + + verifyTaskLaunched("o1") + verifyDeclinedOffer(driver, createOfferId("o2"), true) + } + test("mesos assigns tasks round-robin on offers") { val executorCores = 4 val maxCores = executorCores * 2 diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala index d223af1496a4..f684e16c25f7 100644 --- a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala @@ -30,8 +30,8 @@ class AllStagesResourceSuite extends SparkFunSuite { def getFirstTaskLaunchTime(taskLaunchTimes: Seq[Long]): Option[Date] = { val tasks = new HashMap[Long, TaskUIData] taskLaunchTimes.zipWithIndex.foreach { case (time, idx) => - tasks(idx.toLong) = new TaskUIData( - new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false), None, None) + tasks(idx.toLong) = TaskUIData( + new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false), None) } val stageUiData = new StageUIData() diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index 9ee83b76e71d..1b325801e27f 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -208,16 +208,14 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { } } - test("cannot call lockForWriting while already holding a write lock") { + test("cannot grab a writer lock while already holding a write lock") { withTaskId(0) { assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) blockInfoManager.unlock("block") } withTaskId(1) { assert(blockInfoManager.lockForWriting("block").isDefined) - intercept[IllegalStateException] { - blockInfoManager.lockForWriting("block") - } + assert(blockInfoManager.lockForWriting("block", false).isEmpty) blockInfoManager.assertBlockIsLockedForWriting("block") } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index d14728cb5055..31687e614731 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -27,6 +27,7 @@ import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.concurrent.Eventually._ import org.apache.spark._ +import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.memory.UnifiedMemoryManager import org.apache.spark.network.BlockTransferService import org.apache.spark.network.netty.NettyBlockTransferService @@ -43,7 +44,8 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo private var rpcEnv: RpcEnv = null private var master: BlockManagerMaster = null private val securityMgr = new SecurityManager(conf) - private val mapOutputTracker = new MapOutputTrackerMaster(conf) + private val bcastManager = new BroadcastManager(true, conf, securityMgr) + private val mapOutputTracker = new MapOutputTrackerMaster(conf, bcastManager, true) private val shuffleManager = new SortShuffleManager(conf) // List of block manager created during an unit test, so that all of the them can be stopped diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index db1efaf2a20b..a2580304c4ed 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -33,6 +33,7 @@ import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ import org.apache.spark._ +import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.executor.DataReadMethod import org.apache.spark.memory.UnifiedMemoryManager import org.apache.spark.network.{BlockDataManager, BlockTransferService} @@ -59,7 +60,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE var rpcEnv: RpcEnv = null var master: BlockManagerMaster = null val securityMgr = new SecurityManager(new SparkConf(false)) - val mapOutputTracker = new MapOutputTrackerMaster(new SparkConf(false)) + val bcastManager = new BroadcastManager(true, new SparkConf(false), securityMgr) + val mapOutputTracker = new MapOutputTrackerMaster(new SparkConf(false), bcastManager, true) val shuffleManager = new SortShuffleManager(new SparkConf(false)) // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index ce7d51d1c371..1fa9b28edf4b 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -25,7 +25,8 @@ import org.apache.spark._ import org.apache.spark.{LocalSparkContext, SparkConf, Success} import org.apache.spark.executor._ import org.apache.spark.scheduler._ -import org.apache.spark.util.Utils +import org.apache.spark.ui.jobs.UIData.TaskUIData +import org.apache.spark.util.{AccumulatorContext, Utils} class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with Matchers { @@ -359,4 +360,30 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with assert( stage1Data.taskData.get(1237L).get.metrics.get.shuffleReadMetrics.totalBlocksFetched == 402) } + + test("drop internal and sql accumulators") { + val taskInfo = new TaskInfo(0, 0, 0, 0, "", "", TaskLocality.ANY, false) + val internalAccum = + AccumulableInfo(id = 1, name = Some("internal"), None, None, true, false, None) + val sqlAccum = AccumulableInfo( + id = 2, + name = Some("sql"), + update = None, + value = None, + internal = false, + countFailedValues = false, + metadata = Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) + val userAccum = AccumulableInfo( + id = 3, + name = Some("user"), + update = None, + value = None, + internal = false, + countFailedValues = false, + metadata = None) + taskInfo.accumulables ++= Seq(internalAccum, sqlAccum, userAccum) + + val newTaskInfo = TaskUIData.dropInternalAndSQLAccumulables(taskInfo) + assert(newTaskInfo.accumulables === Seq(userAccum)) + } } diff --git a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala new file mode 100644 index 000000000000..439da1306f5a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala @@ -0,0 +1,165 @@ +/* + * 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 org.apache.spark._ + +class AccumulatorV2Suite extends SparkFunSuite { + + test("LongAccumulator add/avg/sum/count/isZero") { + val acc = new LongAccumulator + assert(acc.isZero) + assert(acc.count == 0) + assert(acc.sum == 0) + assert(acc.avg.isNaN) + + acc.add(0) + assert(!acc.isZero) + assert(acc.count == 1) + assert(acc.sum == 0) + assert(acc.avg == 0.0) + + acc.add(1) + assert(acc.count == 2) + assert(acc.sum == 1) + assert(acc.avg == 0.5) + + // Also test add using non-specialized add function + acc.add(new java.lang.Long(2)) + assert(acc.count == 3) + assert(acc.sum == 3) + assert(acc.avg == 1.0) + + // Test merging + val acc2 = new LongAccumulator + acc2.add(2) + acc.merge(acc2) + assert(acc.count == 4) + assert(acc.sum == 5) + assert(acc.avg == 1.25) + } + + test("DoubleAccumulator add/avg/sum/count/isZero") { + val acc = new DoubleAccumulator + assert(acc.isZero) + assert(acc.count == 0) + assert(acc.sum == 0.0) + assert(acc.avg.isNaN) + + acc.add(0.0) + assert(!acc.isZero) + assert(acc.count == 1) + assert(acc.sum == 0.0) + assert(acc.avg == 0.0) + + acc.add(1.0) + assert(acc.count == 2) + assert(acc.sum == 1.0) + assert(acc.avg == 0.5) + + // Also test add using non-specialized add function + acc.add(new java.lang.Double(2.0)) + assert(acc.count == 3) + assert(acc.sum == 3.0) + assert(acc.avg == 1.0) + + // Test merging + val acc2 = new DoubleAccumulator + acc2.add(2.0) + acc.merge(acc2) + assert(acc.count == 4) + assert(acc.sum == 5.0) + assert(acc.avg == 1.25) + } + + test("ListAccumulator") { + val acc = new ListAccumulator[Double] + assert(acc.value.isEmpty) + assert(acc.isZero) + + acc.add(0.0) + assert(acc.value.contains(0.0)) + assert(!acc.isZero) + + acc.add(new java.lang.Double(1.0)) + + val acc2 = acc.copyAndReset() + assert(acc2.value.isEmpty) + assert(acc2.isZero) + + assert(acc.value.contains(1.0)) + assert(!acc.isZero) + assert(acc.value.size() === 2) + + acc2.add(2.0) + assert(acc2.value.contains(2.0)) + assert(!acc2.isZero) + assert(acc2.value.size() === 1) + + // Test merging + acc.merge(acc2) + assert(acc.value.contains(2.0)) + assert(!acc.isZero) + assert(acc.value.size() === 3) + + val acc3 = acc.copy() + assert(acc3.value.contains(2.0)) + assert(!acc3.isZero) + assert(acc3.value.size() === 3) + + acc3.reset() + assert(acc3.isZero) + assert(acc3.value.isEmpty) + } + + test("LegacyAccumulatorWrapper") { + val acc = new LegacyAccumulatorWrapper("default", AccumulatorParam.StringAccumulatorParam) + assert(acc.value === "default") + assert(!acc.isZero) + + acc.add("foo") + assert(acc.value === "foo") + assert(!acc.isZero) + + acc.add(new java.lang.String("bar")) + + val acc2 = acc.copyAndReset() + assert(acc2.value === "") + assert(acc2.isZero) + + assert(acc.value === "bar") + assert(!acc.isZero) + + acc2.add("baz") + assert(acc2.value === "baz") + assert(!acc2.isZero) + + // Test merging + acc.merge(acc2) + assert(acc.value === "baz") + assert(!acc.isZero) + + val acc3 = acc.copy() + assert(acc3.value === "baz") + assert(!acc3.isZero) + + acc3.reset() + assert(acc3.isZero) + assert(acc3.value === "") + } +} diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index 52428634e520..def0752b46f6 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging import org.apache.spark.unsafe.array.LongArray import org.apache.spark.unsafe.memory.MemoryBlock -import org.apache.spark.util.Benchmark import org.apache.spark.util.collection.Sorter import org.apache.spark.util.random.XORShiftRandom @@ -184,81 +183,4 @@ class RadixSortSuite extends SparkFunSuite with Logging { assert(res1.view == res2.view) } } - - ignore("microbenchmarks") { - val size = 25000000 - val rand = new XORShiftRandom(123) - val benchmark = new Benchmark("radix sort " + size, size) - benchmark.addTimerCase("reference TimSort key prefix array") { timer => - val array = Array.tabulate[Long](size * 2) { i => rand.nextLong } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) - timer.startTiming() - referenceKeyPrefixSort(buf, 0, size, PrefixComparators.BINARY) - timer.stopTiming() - } - benchmark.addTimerCase("reference Arrays.sort") { timer => - val ref = Array.tabulate[Long](size) { i => rand.nextLong } - timer.startTiming() - Arrays.sort(ref) - timer.stopTiming() - } - benchmark.addTimerCase("radix sort one byte") { timer => - val array = new Array[Long](size * 2) - var i = 0 - while (i < size) { - array(i) = rand.nextLong & 0xff - i += 1 - } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) - timer.startTiming() - RadixSort.sort(buf, size, 0, 7, false, false) - timer.stopTiming() - } - benchmark.addTimerCase("radix sort two bytes") { timer => - val array = new Array[Long](size * 2) - var i = 0 - while (i < size) { - array(i) = rand.nextLong & 0xffff - i += 1 - } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) - timer.startTiming() - RadixSort.sort(buf, size, 0, 7, false, false) - timer.stopTiming() - } - benchmark.addTimerCase("radix sort eight bytes") { timer => - val array = new Array[Long](size * 2) - var i = 0 - while (i < size) { - array(i) = rand.nextLong - i += 1 - } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) - timer.startTiming() - RadixSort.sort(buf, size, 0, 7, false, false) - timer.stopTiming() - } - benchmark.addTimerCase("radix sort key prefix array") { timer => - val (_, buf2) = generateKeyPrefixTestData(size, rand.nextLong) - timer.startTiming() - RadixSort.sortKeyPrefixArray(buf2, size, 0, 7, false, false) - timer.stopTiming() - } - benchmark.run - - /** - Running benchmark: radix sort 25000000 - Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 3.13.0-44-generic - Intel(R) Core(TM) i7-4600U CPU @ 2.10GHz - - radix sort 25000000: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - reference TimSort key prefix array 15546 / 15859 1.6 621.9 1.0X - reference Arrays.sort 2416 / 2446 10.3 96.6 6.4X - radix sort one byte 133 / 137 188.4 5.3 117.2X - radix sort two bytes 255 / 258 98.2 10.2 61.1X - radix sort eight bytes 991 / 997 25.2 39.6 15.7X - radix sort key prefix array 1540 / 1563 16.2 61.6 10.1X - */ - } } diff --git a/data/mllib/sample_kmeans_data.txt b/data/mllib/sample_kmeans_data.txt new file mode 100644 index 000000000000..50013776b182 --- /dev/null +++ b/data/mllib/sample_kmeans_data.txt @@ -0,0 +1,6 @@ +0 1:0.0 2:0.0 3:0.0 +1 1:0.1 2:0.1 3:0.1 +2 1:0.2 2:0.2 3:0.2 +3 1:9.0 2:9.0 3:9.0 +4 1:9.1 2:9.1 3:9.1 +5 1:9.2 2:9.2 3:9.2 diff --git a/data/mllib/sample_lda_libsvm_data.txt b/data/mllib/sample_lda_libsvm_data.txt new file mode 100644 index 000000000000..bf118d7d5b20 --- /dev/null +++ b/data/mllib/sample_lda_libsvm_data.txt @@ -0,0 +1,12 @@ +0 1:1 2:2 3:6 4:0 5:2 6:3 7:1 8:1 9:0 10:0 11:3 +1 1:1 2:3 3:0 4:1 5:3 6:0 7:0 8:2 9:0 10:0 11:1 +2 1:1 2:4 3:1 4:0 5:0 6:4 7:9 8:0 9:1 10:2 11:0 +3 1:2 2:1 3:0 4:3 5:0 6:0 7:5 8:0 9:2 10:3 11:9 +4 1:3 2:1 3:1 4:9 5:3 6:0 7:2 8:0 9:0 10:1 11:3 +5 1:4 2:2 3:0 4:3 5:4 6:5 7:1 8:1 9:1 10:4 11:0 +6 1:2 2:1 3:0 4:3 5:0 6:0 7:5 8:0 9:2 10:2 11:9 +7 1:1 2:1 3:1 4:9 5:2 6:1 7:2 8:0 9:0 10:1 11:3 +8 1:4 2:4 3:0 4:3 5:4 6:2 7:1 8:3 9:0 10:0 11:0 +9 1:2 2:8 3:2 4:0 5:3 6:0 7:2 8:0 9:2 10:7 11:2 +10 1:1 2:1 3:1 4:9 5:0 6:2 7:2 8:0 9:0 10:3 11:3 +11 1:4 2:1 3:0 4:0 5:4 6:5 7:1 8:3 9:0 10:1 11:0 diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py index ee72da4df065..b28e7a427b8f 100755 --- a/dev/audit-release/audit_release.py +++ b/dev/audit-release/audit_release.py @@ -116,7 +116,7 @@ def ensure_path_not_present(path): # dependencies within those projects. modules = [ "spark-core", "spark-mllib", "spark-streaming", "spark-repl", - "spark-graphx", "spark-streaming-flume", "spark-streaming-kafka", + "spark-graphx", "spark-streaming-flume", "spark-streaming-kafka-0-8", "spark-catalyst", "spark-sql", "spark-hive", "spark-streaming-kinesis-asl" ] modules = map(lambda m: "%s_%s" % (m, SCALA_BINARY_VERSION), modules) diff --git a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala index f69d46cd17d0..8cbfb9cd41b3 100644 --- a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala +++ b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala @@ -33,7 +33,9 @@ object SparkSqlExample { case None => new SparkConf().setAppName("Simple Sql App") } val sc = new SparkContext(conf) - val sparkSession = SparkSession.withHiveSupport(sc) + val sparkSession = SparkSession.builder + .enableHiveSupport() + .getOrCreate() import sparkSession._ sql("DROP TABLE IF EXISTS src") @@ -41,14 +43,14 @@ object SparkSqlExample { sql("LOAD DATA LOCAL INPATH 'data.txt' INTO TABLE src") val results = sql("FROM src SELECT key, value WHERE key >= 0 AND KEY < 5").collect() results.foreach(println) - + def test(f: => Boolean, failureMsg: String) = { if (!f) { println(failureMsg) System.exit(-1) } } - + test(results.size == 5, "Unexpected number of selected elements: " + results) println("Test succeeded") sc.stop() diff --git a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala index 69c1154dc095..10026314ef7a 100644 --- a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala +++ b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala @@ -41,7 +41,7 @@ object SparkSqlExample { import sqlContext._ val people = sc.makeRDD(1 to 100, 10).map(x => Person(s"Name$x", x)).toDF() - people.registerTempTable("people") + people.createOrReplaceTempView("people") val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") val teenagerNames = teenagers.map(t => "Name: " + t(0)).collect() teenagerNames.foreach(println) @@ -52,7 +52,7 @@ object SparkSqlExample { System.exit(-1) } } - + test(teenagerNames.size == 7, "Unexpected number of selected elements: " + teenagerNames) println("Test succeeded") sc.stop() diff --git a/dev/checkstyle-suppressions.xml b/dev/checkstyle-suppressions.xml index a1a88ac8cdac..bfc2e734e517 100644 --- a/dev/checkstyle-suppressions.xml +++ b/dev/checkstyle-suppressions.xml @@ -36,4 +36,10 @@ files="src/test/java/org/apache/spark/sql/hive/test/Complex.java"/> + + + diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index a4ef9a9af293..91b333fcae4f 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -1,14 +1,13 @@ JavaEWAH-0.3.2.jar RoaringBitmap-0.5.11.jar ST4-4.0.4.jar -activation-1.1.jar antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.2-1.jar aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar -asm-3.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -31,7 +30,7 @@ commons-configuration-1.6.jar commons-dbcp-1.4.jar commons-digester-1.8.jar commons-httpclient-3.1.jar -commons-io-2.1.jar +commons-io-2.4.jar commons-lang-2.6.jar commons-lang3-3.3.2.jar commons-logging-1.1.3.jar @@ -47,14 +46,8 @@ curator-recipes-2.4.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.10.1.1.jar +derby-10.11.1.1.jar eigenbase-properties-1.1.5.jar -gmbal-api-only-3.0.0-b023.jar -grizzly-framework-2.1.2.jar -grizzly-http-2.1.2.jar -grizzly-http-server-2.1.2.jar -grizzly-http-servlet-2.1.2.jar -grizzly-rcm-2.1.2.jar guava-14.0.1.jar guice-3.0.jar guice-servlet-3.0.jar @@ -73,46 +66,45 @@ hadoop-yarn-client-2.2.0.jar hadoop-yarn-common-2.2.0.jar hadoop-yarn-server-common-2.2.0.jar hadoop-yarn-server-web-proxy-2.2.0.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar jackson-databind-2.5.3.jar -jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar jackson-module-scala_2.11-2.5.3.jar -jackson-xc-1.9.13.jar janino-2.7.8.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar javax.inject-1.jar -javax.servlet-3.0.0.v201112011016.jar -javax.servlet-3.1.jar -javax.servlet-api-3.0.1.jar +javax.inject-2.4.0-b34.jar +javax.servlet-api-3.1.0.jar +javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar -jaxb-api-2.2.2.jar -jaxb-impl-2.2.3-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-1.9.jar -jersey-core-1.9.jar -jersey-grizzly2-1.9.jar -jersey-guice-1.9.jar -jersey-json-1.9.jar -jersey-server-1.9.jar -jersey-test-framework-core-1.9.jar -jersey-test-framework-grizzly2-1.9.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar jets3t-0.7.1.jar -jettison-1.1.jar jetty-util-6.1.26.jar jline-2.12.jar joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar json-20090211.jar -json4s-ast_2.11-3.2.10.jar -json4s-core_2.11-3.2.10.jar -json4s-jackson_2.11-3.2.10.jar +json4s-ast_2.11-3.2.11.jar +json4s-core_2.11-3.2.11.jar +json4s-jackson_2.11-3.2.11.jar jsr305-1.3.9.jar jta-1.1.jar jtransforms-2.4.0.jar @@ -123,7 +115,6 @@ libfb303-0.9.2.jar libthrift-0.9.2.jar log4j-1.2.17.jar lz4-1.3.0.jar -management-api-3.0.0-b012.jar mesos-0.21.1-shaded-protobuf.jar metrics-core-3.1.2.jar metrics-graphite-3.1.2.jar @@ -135,6 +126,7 @@ netty-all-4.0.29.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar paranamer-2.6.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar @@ -148,7 +140,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.2.jar +py4j-0.10.1.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar @@ -162,12 +154,12 @@ snappy-0.2.jar snappy-java-1.1.2.4.jar spire-macros_2.11-0.7.4.jar spire_2.11-0.7.4.jar -stax-api-1.0-2.jar stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar -univocity-parsers-2.0.2.jar +univocity-parsers-2.1.0.jar +validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar xz-1.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 19c8fad984b3..61ed4c0889b8 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -6,9 +6,9 @@ antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.2-1.jar aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar -asm-3.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -48,7 +48,7 @@ curator-recipes-2.4.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.10.1.1.jar +derby-10.11.1.1.jar eigenbase-properties-1.1.5.jar guava-14.0.1.jar guice-3.0.jar @@ -68,32 +68,38 @@ hadoop-yarn-client-2.3.0.jar hadoop-yarn-common-2.3.0.jar hadoop-yarn-server-common-2.3.0.jar hadoop-yarn-server-web-proxy-2.3.0.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar jackson-databind-2.5.3.jar -jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar jackson-module-scala_2.11-2.5.3.jar -jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar javax.inject-1.jar -javax.servlet-3.0.0.v201112011016.jar +javax.inject-2.4.0-b34.jar +javax.servlet-api-3.1.0.jar +javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar -jaxb-impl-2.2.3-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-core-1.9.jar -jersey-guice-1.9.jar -jersey-json-1.9.jar -jersey-server-1.9.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar jets3t-0.9.3.jar -jettison-1.1.jar jetty-6.1.26.jar jetty-util-6.1.26.jar jline-2.12.jar @@ -101,9 +107,9 @@ joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar json-20090211.jar -json4s-ast_2.11-3.2.10.jar -json4s-core_2.11-3.2.10.jar -json4s-jackson_2.11-3.2.10.jar +json4s-ast_2.11-3.2.11.jar +json4s-core_2.11-3.2.11.jar +json4s-jackson_2.11-3.2.11.jar jsr305-1.3.9.jar jta-1.1.jar jtransforms-2.4.0.jar @@ -127,6 +133,7 @@ netty-all-4.0.29.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar paranamer-2.6.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar @@ -140,7 +147,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.2.jar +py4j-0.10.1.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar @@ -159,7 +166,8 @@ stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar -univocity-parsers-2.0.2.jar +univocity-parsers-2.1.0.jar +validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar xz-1.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index c2365f9cc562..fb014921765f 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -6,9 +6,9 @@ antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.2-1.jar aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar -asm-3.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -48,7 +48,7 @@ curator-recipes-2.4.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.10.1.1.jar +derby-10.11.1.1.jar eigenbase-properties-1.1.5.jar guava-14.0.1.jar guice-3.0.jar @@ -68,33 +68,38 @@ hadoop-yarn-client-2.4.0.jar hadoop-yarn-common-2.4.0.jar hadoop-yarn-server-common-2.4.0.jar hadoop-yarn-server-web-proxy-2.4.0.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar jackson-databind-2.5.3.jar -jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar jackson-module-scala_2.11-2.5.3.jar -jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar javax.inject-1.jar -javax.servlet-3.0.0.v201112011016.jar +javax.inject-2.4.0-b34.jar +javax.servlet-api-3.1.0.jar +javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar -jaxb-impl-2.2.3-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-1.9.jar -jersey-core-1.9.jar -jersey-guice-1.9.jar -jersey-json-1.9.jar -jersey-server-1.9.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar jets3t-0.9.3.jar -jettison-1.1.jar jetty-6.1.26.jar jetty-util-6.1.26.jar jline-2.12.jar @@ -102,9 +107,9 @@ joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar json-20090211.jar -json4s-ast_2.11-3.2.10.jar -json4s-core_2.11-3.2.10.jar -json4s-jackson_2.11-3.2.10.jar +json4s-ast_2.11-3.2.11.jar +json4s-core_2.11-3.2.11.jar +json4s-jackson_2.11-3.2.11.jar jsr305-1.3.9.jar jta-1.1.jar jtransforms-2.4.0.jar @@ -128,6 +133,7 @@ netty-all-4.0.29.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar paranamer-2.6.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar @@ -141,7 +147,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.2.jar +py4j-0.10.1.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar @@ -160,7 +166,8 @@ stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar -univocity-parsers-2.0.2.jar +univocity-parsers-2.1.0.jar +validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar xz-1.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 0d8afd19c561..0baf4e84fff0 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -6,13 +6,13 @@ antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.2-1.jar aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar apacheds-i18n-2.0.0-M15.jar apacheds-kerberos-codec-2.0.0-M15.jar api-asn1-api-1.0.0-M20.jar api-util-1.0.0-M20.jar arpack_combined_all-0.1.jar -asm-3.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -52,7 +52,7 @@ curator-recipes-2.6.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.10.1.1.jar +derby-10.11.1.1.jar eigenbase-properties-1.1.5.jar gson-2.2.4.jar guava-14.0.1.jar @@ -73,9 +73,12 @@ hadoop-yarn-client-2.6.0.jar hadoop-yarn-common-2.6.0.jar hadoop-yarn-server-common-2.6.0.jar hadoop-yarn-server-web-proxy-2.6.0.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar htrace-core-3.0.4.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar @@ -87,20 +90,24 @@ jackson-module-scala_2.11-2.5.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar javax.inject-1.jar -javax.servlet-3.0.0.v201112011016.jar +javax.inject-2.4.0-b34.jar +javax.servlet-api-3.1.0.jar +javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar -jaxb-impl-2.2.3-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-1.9.jar -jersey-core-1.9.jar -jersey-guice-1.9.jar -jersey-json-1.9.jar -jersey-server-1.9.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar jets3t-0.9.3.jar -jettison-1.1.jar jetty-6.1.26.jar jetty-util-6.1.26.jar jline-2.12.jar @@ -108,9 +115,9 @@ joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar json-20090211.jar -json4s-ast_2.11-3.2.10.jar -json4s-core_2.11-3.2.10.jar -json4s-jackson_2.11-3.2.10.jar +json4s-ast_2.11-3.2.11.jar +json4s-core_2.11-3.2.11.jar +json4s-jackson_2.11-3.2.11.jar jsr305-1.3.9.jar jta-1.1.jar jtransforms-2.4.0.jar @@ -134,6 +141,7 @@ netty-all-4.0.29.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar paranamer-2.6.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar @@ -147,7 +155,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.2.jar +py4j-0.10.1.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar @@ -166,7 +174,8 @@ stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar -univocity-parsers-2.0.2.jar +univocity-parsers-2.1.0.jar +validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xercesImpl-2.9.1.jar xmlenc-0.52.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index f4274a9441a0..8be218cd68d9 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -6,13 +6,13 @@ antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.2-1.jar aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar apacheds-i18n-2.0.0-M15.jar apacheds-kerberos-codec-2.0.0-M15.jar api-asn1-api-1.0.0-M20.jar api-util-1.0.0-M20.jar arpack_combined_all-0.1.jar -asm-3.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -52,7 +52,7 @@ curator-recipes-2.6.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.10.1.1.jar +derby-10.11.1.1.jar eigenbase-properties-1.1.5.jar gson-2.2.4.jar guava-14.0.1.jar @@ -73,9 +73,12 @@ hadoop-yarn-client-2.7.0.jar hadoop-yarn-common-2.7.0.jar hadoop-yarn-server-common-2.7.0.jar hadoop-yarn-server-web-proxy-2.7.0.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar htrace-core-3.1.0-incubating.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar @@ -87,20 +90,24 @@ jackson-module-scala_2.11-2.5.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar javax.inject-1.jar -javax.servlet-3.0.0.v201112011016.jar +javax.inject-2.4.0-b34.jar +javax.servlet-api-3.1.0.jar +javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar -jaxb-impl-2.2.3-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-1.9.jar -jersey-core-1.9.jar -jersey-guice-1.9.jar -jersey-json-1.9.jar -jersey-server-1.9.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar jets3t-0.9.3.jar -jettison-1.1.jar jetty-6.1.26.jar jetty-util-6.1.26.jar jline-2.12.jar @@ -108,9 +115,9 @@ joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar json-20090211.jar -json4s-ast_2.11-3.2.10.jar -json4s-core_2.11-3.2.10.jar -json4s-jackson_2.11-3.2.10.jar +json4s-ast_2.11-3.2.11.jar +json4s-core_2.11-3.2.11.jar +json4s-jackson_2.11-3.2.11.jar jsp-api-2.1.jar jsr305-1.3.9.jar jta-1.1.jar @@ -135,6 +142,7 @@ netty-all-4.0.29.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar paranamer-2.6.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar @@ -148,7 +156,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.2.jar +py4j-0.10.1.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar @@ -167,7 +175,8 @@ stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar -univocity-parsers-2.0.2.jar +univocity-parsers-2.1.0.jar +validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xercesImpl-2.9.1.jar xmlenc-0.52.jar diff --git a/dev/run-tests.py b/dev/run-tests.py index 291f821c7f80..7b3269752bac 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -336,7 +336,7 @@ def build_spark_sbt(hadoop_version): # Enable all of the profiles for the build: build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags sbt_goals = ["package", - "streaming-kafka-assembly/assembly", + "streaming-kafka-0-8-assembly/assembly", "streaming-flume-assembly/assembly", "streaming-kinesis-asl-assembly/assembly"] profiles_and_goals = build_profiles + sbt_goals diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 564092864395..0d6aa7422a53 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -92,10 +92,17 @@ def __ne__(self, other): def __hash__(self): return hash(self.name) +tags = Module( + name="tags", + dependencies=[], + source_file_regexes=[ + "common/tags/", + ] +) catalyst = Module( name="catalyst", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "sql/catalyst/", ], @@ -165,7 +172,7 @@ def __hash__(self): sketch = Module( name="sketch", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "common/sketch/", ], @@ -177,7 +184,7 @@ def __hash__(self): graphx = Module( name="graphx", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "graphx/", ], @@ -189,7 +196,7 @@ def __hash__(self): streaming = Module( name="streaming", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "streaming", ], @@ -205,7 +212,7 @@ def __hash__(self): # fail other PRs. streaming_kinesis_asl = Module( name="streaming-kinesis-asl", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "external/kinesis-asl/", "external/kinesis-asl-assembly/", @@ -223,14 +230,14 @@ def __hash__(self): streaming_kafka = Module( - name="streaming-kafka", + name="streaming-kafka-0-8", dependencies=[streaming], source_file_regexes=[ - "external/kafka", - "external/kafka-assembly", + "external/kafka-0-8", + "external/kafka-0-8-assembly", ], sbt_test_goals=[ - "streaming-kafka/test", + "streaming-kafka-0-8/test", ] ) @@ -270,7 +277,7 @@ def __hash__(self): mllib_local = Module( name="mllib-local", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "mllib-local", ], @@ -406,6 +413,7 @@ def __hash__(self): "pyspark.ml.feature", "pyspark.ml.classification", "pyspark.ml.clustering", + "pyspark.ml.linalg.__init__", "pyspark.ml.recommendation", "pyspark.ml.regression", "pyspark.ml.tuning", diff --git a/docs/_config.yml b/docs/_config.yml index 8bdc68aeeac7..c0a3be7f8208 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,7 +14,7 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.0.0-SNAPSHOT +SPARK_VERSION: 2.0.0-preview SPARK_VERSION_SHORT: 2.0.0 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.7" diff --git a/docs/building-spark.md b/docs/building-spark.md index fec442af95e1..2c987cf8346e 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -7,48 +7,18 @@ redirect_from: "building-with-maven.html" * This will become a table of contents (this text will be scraped). {:toc} -Building Spark using Maven requires Maven 3.3.9 or newer and Java 7+. -The Spark build can supply a suitable Maven binary; see below. - -# Building with `build/mvn` - -Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](http://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows: - -{% highlight bash %} -build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package -{% endhighlight %} - -Other build examples can be found below. - -**Note:** When building on an encrypted filesystem (if your home directory is encrypted, for example), then the Spark build might fail with a "Filename too long" error. As a workaround, add the following in the configuration args of the `scala-maven-plugin` in the project `pom.xml`: - - -Xmax-classfile-name - 128 - -and in `project/SparkBuild.scala` add: - - scalacOptions in Compile ++= Seq("-Xmax-classfile-name", "128"), - -to the `sharedSettings` val. See also [this PR](https://github.com/apache/spark/pull/2883/files) if you are unsure of where to add these lines. +# Building Apache Spark -# Building a Runnable Distribution +## Apache Maven -To create a Spark distribution like those distributed by the -[Spark Downloads](http://spark.apache.org/downloads.html) page, and that is laid out so as -to be runnable, use `./dev/make-distribution.sh` in the project root directory. It can be configured -with Maven profile settings and so on like the direct Maven build. Example: - - ./dev/make-distribution.sh --name custom-spark --tgz -Psparkr -Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn - -For more information on usage, run `./dev/make-distribution.sh --help` +The Maven-based build is the build of reference for Apache Spark. +Building Spark using Maven requires Maven 3.3.9 or newer and Java 7+. -# Setting up Maven's Memory Usage +### Setting up Maven's Memory Usage You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`. We recommend the following settings: -{% highlight bash %} -export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" -{% endhighlight %} + export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" If you don't run this, you may see errors like the following: @@ -65,7 +35,26 @@ You can fix this by setting the `MAVEN_OPTS` variable as discussed before. * For Java 8 and above this step is not required. * If using `build/mvn` with no `MAVEN_OPTS` set, the script will automate this for you. -# Specifying the Hadoop Version +### build/mvn + +Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](http://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows: + + ./build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package + +Other build examples can be found below. + +## Building a Runnable Distribution + +To create a Spark distribution like those distributed by the +[Spark Downloads](http://spark.apache.org/downloads.html) page, and that is laid out so as +to be runnable, use `./dev/make-distribution.sh` in the project root directory. It can be configured +with Maven profile settings and so on like the direct Maven build. Example: + + ./dev/make-distribution.sh --name custom-spark --tgz -Psparkr -Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn + +For more information on usage, run `./dev/make-distribution.sh --help` + +## Specifying the Hadoop Version Because HDFS is not protocol-compatible across versions, if you want to read from HDFS, you'll need to build Spark against the specific HDFS version in your environment. You can do this through the `hadoop.version` property. If unset, Spark will build against Hadoop 2.2.0 by default. Note that certain build profiles are required for particular Hadoop versions: @@ -87,72 +76,63 @@ You can enable the `yarn` profile and optionally set the `yarn.version` property Examples: -{% highlight bash %} + # Apache Hadoop 2.2.X + ./build/mvn -Pyarn -Phadoop-2.2 -DskipTests clean package -# Apache Hadoop 2.2.X -mvn -Pyarn -Phadoop-2.2 -DskipTests clean package + # Apache Hadoop 2.3.X + ./build/mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -DskipTests clean package -# Apache Hadoop 2.3.X -mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -DskipTests clean package + # Apache Hadoop 2.4.X or 2.5.X + ./build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=VERSION -DskipTests clean package -# Apache Hadoop 2.4.X or 2.5.X -mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=VERSION -DskipTests clean package + # Apache Hadoop 2.6.X + ./build/mvn -Pyarn -Phadoop-2.6 -Dhadoop.version=2.6.0 -DskipTests clean package -# Apache Hadoop 2.6.X -mvn -Pyarn -Phadoop-2.6 -Dhadoop.version=2.6.0 -DskipTests clean package + # Apache Hadoop 2.7.X and later + ./build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=VERSION -DskipTests clean package -# Apache Hadoop 2.7.X and later -mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=VERSION -DskipTests clean package + # Different versions of HDFS and YARN. + ./build/mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=2.2.0 -DskipTests clean package -# Different versions of HDFS and YARN. -mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=2.2.0 -DskipTests clean package -{% endhighlight %} +## Building With Hive and JDBC Support -# Building With Hive and JDBC Support To enable Hive integration for Spark SQL along with its JDBC server and CLI, add the `-Phive` and `Phive-thriftserver` profiles to your existing build options. By default Spark will build with Hive 1.2.1 bindings. -{% highlight bash %} -# Apache Hadoop 2.4.X with Hive 1.2.1 support -mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-thriftserver -DskipTests clean package -{% endhighlight %} -# Building for Scala 2.10 -To produce a Spark package compiled with Scala 2.10, use the `-Dscala-2.10` property: - - ./dev/change-scala-version.sh 2.10 - mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package - -# Spark Tests in Maven + # Apache Hadoop 2.4.X with Hive 1.2.1 support + ./build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-thriftserver -DskipTests clean package -Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). - -Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: +## Packaging without Hadoop Dependencies for YARN - mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive -Phive-thriftserver clean package - mvn -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test +The assembly directory produced by `mvn package` will, by default, include all of Spark's +dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this +causes multiple versions of these to appear on executor classpaths: the version packaged in +the Spark assembly and the version on each node, included with `yarn.application.classpath`. +The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, +like ZooKeeper and Hadoop itself. -The ScalaTest plugin also supports running only a specific test suite as follows: +## Building for Scala 2.10 +To produce a Spark package compiled with Scala 2.10, use the `-Dscala-2.10` property: - mvn -Dhadoop.version=... -DwildcardSuites=org.apache.spark.repl.ReplSuite test + ./dev/change-scala-version.sh 2.10 + ./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package -# Building submodules individually +## Building submodules individually It's possible to build Spark sub-modules using the `mvn -pl` option. For instance, you can build the Spark Streaming module using: -{% highlight bash %} -mvn -pl :spark-streaming_2.11 clean install -{% endhighlight %} + ./build/mvn -pl :spark-streaming_2.11 clean install where `spark-streaming_2.11` is the `artifactId` as defined in `streaming/pom.xml` file. -# Continuous Compilation +## Continuous Compilation We use the scala-maven-plugin which supports incremental and continuous compilation. E.g. - mvn scala:cc + ./build/mvn scala:cc should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. A couple of gotchas to note: @@ -167,34 +147,25 @@ the `spark-parent` module). Thus, the full flow for running continuous-compilation of the `core` submodule may look more like: - $ mvn install + $ ./build/mvn install $ cd core - $ mvn scala:cc - -# Building Spark with IntelliJ IDEA or Eclipse - -For help in setting up IntelliJ IDEA or Eclipse for Spark development, and troubleshooting, refer to the -[wiki page for IDE setup](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools#UsefulDeveloperTools-IDESetup). - -# Running Java 8 Test Suites - -Running only Java 8 tests and nothing else. + $ ../build/mvn scala:cc - mvn install -DskipTests - mvn -pl :java8-tests_2.11 test +## Speeding up Compilation with Zinc -or - - sbt java8-tests/test - -Java 8 tests are automatically enabled when a Java 8 JDK is detected. -If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. - -# Packaging without Hadoop Dependencies for YARN +[Zinc](https://github.com/typesafehub/zinc) is a long-running server version of SBT's incremental +compiler. When run locally as a background process, it speeds up builds of Scala-based projects +like Spark. Developers who regularly recompile Spark with Maven will be the most interested in +Zinc. The project site gives instructions for building and running `zinc`; OS X users can +install it using `brew install zinc`. -The assembly directory produced by `mvn package` will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with `yarn.application.classpath`. The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. +If using the `build/mvn` package `zinc` will automatically be downloaded and leveraged for all +builds. This process will auto-start after the first time `build/mvn` is called and bind to port +3030 unless the `ZINC_PORT` environment variable is set. The `zinc` process can subsequently be +shut down at any time by running `build/zinc-/bin/zinc -shutdown` and will automatically +restart whenever `build/mvn` is called. -# Building with SBT +## Building with SBT Maven is the official build tool recommended for packaging Spark, and is the *build of reference*. But SBT is supported for day-to-day development since it can provide much faster iterative @@ -203,38 +174,111 @@ compilation. More advanced developers may wish to use SBT. The SBT build is derived from the Maven POM files, and so the same Maven profiles and variables can be set to control the SBT build. For example: - build/sbt -Pyarn -Phadoop-2.3 package + ./build/sbt -Pyarn -Phadoop-2.3 package To avoid the overhead of launching sbt each time you need to re-compile, you can launch sbt in interactive mode by running `build/sbt`, and then run all build commands at the command prompt. For more recommendations on reducing build time, refer to the [wiki page](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools#UsefulDeveloperTools-ReducingBuildTimes). -# Testing with SBT +## Encrypted Filesystems + +When building on an encrypted filesystem (if your home directory is encrypted, for example), then the Spark build might fail with a "Filename too long" error. As a workaround, add the following in the configuration args of the `scala-maven-plugin` in the project `pom.xml`: + + -Xmax-classfile-name + 128 + +and in `project/SparkBuild.scala` add: + + scalacOptions in Compile ++= Seq("-Xmax-classfile-name", "128"), + +to the `sharedSettings` val. See also [this PR](https://github.com/apache/spark/pull/2883/files) if you are unsure of where to add these lines. + +## IntelliJ IDEA or Eclipse + +For help in setting up IntelliJ IDEA or Eclipse for Spark development, and troubleshooting, refer to the +[wiki page for IDE setup](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools#UsefulDeveloperTools-IDESetup). + + +# Running Tests + +Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). + +Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: + + ./build/mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive -Phive-thriftserver clean package + ./build/mvn -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test + +The ScalaTest plugin also supports running only a specific Scala test suite as follows: + + ./build/mvn -P... -Dtest=none -DwildcardSuites=org.apache.spark.repl.ReplSuite test + ./build/mvn -P... -Dtest=none -DwildcardSuites=org.apache.spark.repl.* test + +or a Java test: + + ./build/mvn test -P... -DwildcardSuites=none -Dtest=org.apache.spark.streaming.JavaAPISuite + +## Testing with SBT Some of the tests require Spark to be packaged first, so always run `build/sbt package` the first time. The following is an example of a correct (build, test) sequence: - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver package - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test + ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver package + ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test To run only a specific test suite as follows: - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.ReplSuite" + ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.ReplSuite" + ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.*" To run test suites of a specific sub project as follows: - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver core/test + ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver core/test -# Speeding up Compilation with Zinc +## Running Java 8 Test Suites -[Zinc](https://github.com/typesafehub/zinc) is a long-running server version of SBT's incremental -compiler. When run locally as a background process, it speeds up builds of Scala-based projects -like Spark. Developers who regularly recompile Spark with Maven will be the most interested in -Zinc. The project site gives instructions for building and running `zinc`; OS X users can -install it using `brew install zinc`. +Running only Java 8 tests and nothing else. -If using the `build/mvn` package `zinc` will automatically be downloaded and leveraged for all -builds. This process will auto-start after the first time `build/mvn` is called and bind to port -3030 unless the `ZINC_PORT` environment variable is set. The `zinc` process can subsequently be -shut down at any time by running `build/zinc-/bin/zinc -shutdown` and will automatically -restart whenever `build/mvn` is called. + ./build/mvn install -DskipTests + ./build/mvn -pl :java8-tests_2.11 test + +or + + ./build/sbt java8-tests/test + +Java 8 tests are automatically enabled when a Java 8 JDK is detected. +If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. + +## PySpark Tests with Maven + +If you are building PySpark and wish to run the PySpark tests you will need to build Spark with Hive support. + + ./build/mvn -DskipTests clean package -Phive + ./python/run-tests + +The run-tests script also can be limited to a specific Python version or a specific module + + ./python/run-tests --python-executables=python --modules=pyspark-sql + +**Note:** You can also run Python tests with an sbt build, provided you build Spark with Hive support. + +## Running R Tests + +To run the SparkR tests you will need to install the R package `testthat` +(run `install.packages(testthat)` from R shell). You can run just the SparkR tests using +the command: + + ./R/run-tests.sh + +## Running Docker-based Integration Test Suites + +In order to run Docker integration tests, you have to install the `docker` engine on your box. +The instructions for installation can be found at [the Docker site](https://docs.docker.com/engine/installation/). +Once installed, the `docker` service needs to be started, if not already running. +On Linux, this can be done by `sudo service docker start`. + + ./build/mvn install -DskipTests + ./build/mvn -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.11 + +or + + ./build/sbt docker-integration-tests/test diff --git a/docs/configuration.md b/docs/configuration.md index 6512e16faf4c..d23f0fe1a1c4 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -296,7 +296,7 @@ Apart from these, the following properties are also available, and may be useful spark.executor.logs.rolling.maxSize (none) - Set the max size of the file by which the executor logs will be rolled over. + Set the max size of the file in bytes by which the executor logs will be rolled over. Rolling is disabled by default. See spark.executor.logs.rolling.maxRetainedFiles for automatic cleaning of old logs. @@ -308,7 +308,7 @@ Apart from these, the following properties are also available, and may be useful Set the strategy of rolling of executor logs. By default it is disabled. It can be set to "time" (time-based rolling) or "size" (size-based rolling). For "time", use spark.executor.logs.rolling.time.interval to set the rolling interval. - For "size", use spark.executor.logs.rolling.size.maxBytes to set + For "size", use spark.executor.logs.rolling.maxSize to set the maximum file size for rolling. @@ -1231,7 +1231,7 @@ Apart from these, the following properties are also available, and may be useful spark.acls.enable false - Whether Spark acls should are enabled. If enabled, this checks to see if the user has + Whether Spark acls should be enabled. If enabled, this checks to see if the user has access permissions to view or modify the job. Note this requires the user to be known, so if the user comes across as null no checks are done. Filters can be used with the UI to authenticate and set the user. @@ -1243,8 +1243,33 @@ Apart from these, the following properties are also available, and may be useful Comma separated list of users/administrators that have view and modify access to all Spark jobs. This can be used if you run on a shared cluster and have a set of administrators or devs who - help debug when things work. Putting a "*" in the list means any user can have the privilege - of admin. + help debug when things do not work. Putting a "*" in the list means any user can have the + privilege of admin. + + + + spark.admin.acls.groups + Empty + + Comma separated list of groups that have view and modify access to all Spark jobs. + This can be used if you have a set of administrators or developers who help maintain and debug + the underlying infrastructure. Putting a "*" in the list means any user in any group can have + the privilege of admin. The user groups are obtained from the instance of the groups mapping + provider specified by spark.user.groups.mapping. Check the entry + spark.user.groups.mapping for more details. + + + + spark.user.groups.mapping + org.apache.spark.security.ShellBasedGroupsMappingProvider + + The list of groups for a user are determined by a group mapping service defined by the trait + org.apache.spark.security.GroupMappingServiceProvider which can configured by this property. + A default unix shell based implementation is provided org.apache.spark.security.ShellBasedGroupsMappingProvider + which can be specified to resolve a list of groups for a user. + Note: This implementation supports only a Unix/Linux based environment. Windows environment is + currently not supported. However, a new platform/protocol can be supported by implementing + the trait org.apache.spark.security.GroupMappingServiceProvider. @@ -1305,6 +1330,18 @@ Apart from these, the following properties are also available, and may be useful the list means any user can have access to modify it. + + spark.modify.acls.groups + Empty + + Comma separated list of groups that have modify access to the Spark job. This can be used if you + have a set of administrators or developers from the same team to have access to control the job. + Putting a "*" in the list means any user in any group has the access to modify the Spark job. + The user groups are obtained from the instance of the groups mapping provider specified by + spark.user.groups.mapping. Check the entry spark.user.groups.mapping + for more details. + + spark.ui.filters None @@ -1328,6 +1365,18 @@ Apart from these, the following properties are also available, and may be useful have view access to this Spark job. + + spark.ui.view.acls.groups + Empty + + Comma separated list of groups that have view access to the Spark web ui to view the Spark Job + details. This can be used if you have a set of administrators or developers or users who can + monitor the Spark job submitted. Putting a "*" in the list means any user in any group can view + the Spark job details on the Spark web ui. The user groups are obtained from the instance of the + groups mapping provider specified by spark.user.groups.mapping. Check the entry + spark.user.groups.mapping for more details. + + #### Encryption diff --git a/docs/index.md b/docs/index.md index 20eab567a50d..7157afc411bc 100644 --- a/docs/index.md +++ b/docs/index.md @@ -24,8 +24,8 @@ Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). It's easy locally on one machine --- all you need is to have `java` installed on your system `PATH`, or the `JAVA_HOME` environment variable pointing to a Java installation. -Spark runs on Java 7+, Python 2.6+ and R 3.1+. For the Scala API, Spark {{site.SPARK_VERSION}} uses -Scala {{site.SCALA_BINARY_VERSION}}. You will need to use a compatible Scala version +Spark runs on Java 7+, Python 2.6+/3.4+ and R 3.1+. For the Scala API, Spark {{site.SPARK_VERSION}} +uses Scala {{site.SCALA_BINARY_VERSION}}. You will need to use a compatible Scala version ({{site.SCALA_BINARY_VERSION}}.x). # Running the Examples and Shell diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index eaf4f6d84336..f1a21f436f13 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -236,9 +236,9 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classificat Multilayer perceptron classifier (MLPC) is a classifier based on the [feedforward artificial neural network](https://en.wikipedia.org/wiki/Feedforward_neural_network). MLPC consists of multiple layers of nodes. -Each layer is fully connected to the next layer in the network. Nodes in the input layer represent the input data. All other nodes maps inputs to the outputs -by performing linear combination of the inputs with the node's weights `$\wv$` and bias `$\bv$` and applying an activation function. -It can be written in matrix form for MLPC with `$K+1$` layers as follows: +Each layer is fully connected to the next layer in the network. Nodes in the input layer represent the input data. All other nodes map inputs to outputs +by a linear combination of the inputs with the node's weights `$\wv$` and bias `$\bv$` and applying an activation function. +This can be written in matrix form for MLPC with `$K+1$` layers as follows: `\[ \mathrm{y}(\x) = \mathrm{f_K}(...\mathrm{f_2}(\wv_2^T\mathrm{f_1}(\wv_1^T \x+b_1)+b_2)...+b_K) \]` @@ -252,7 +252,7 @@ Nodes in the output layer use softmax function: \]` The number of nodes `$N$` in the output layer corresponds to the number of classes. -MLPC employs backpropagation for learning the model. We use logistic loss function for optimization and L-BFGS as optimization routine. +MLPC employs backpropagation for learning the model. We use the logistic loss function for optimization and L-BFGS as an optimization routine. **Example** @@ -300,13 +300,20 @@ Refer to the [Java API docs](api/java/org/apache/spark/ml/classification/OneVsRe {% include_example java/org/apache/spark/examples/ml/JavaOneVsRestExample.java %} + +
    + +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classification.OneVsRest) for more details. + +{% include_example python/ml/one_vs_rest_example.py %} +
    ## Naive Bayes -[Naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier) are a family of simple +[Naive Bayes classifiers](http://en.wikipedia.org/wiki/Naive_Bayes_classifier) are a family of simple probabilistic classifiers based on applying Bayes' theorem with strong (naive) independence -assumptions between the features. The spark.ml implementation currently supports both [multinomial +assumptions between the features. The `spark.ml` implementation currently supports both [multinomial naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html) and [Bernoulli naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html). More information can be found in the section on [Naive Bayes in MLlib](mllib-naive-bayes.html#naive-bayes-sparkmllib). @@ -475,11 +482,11 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.regression. In `spark.ml`, we implement the [Accelerated failure time (AFT)](https://en.wikipedia.org/wiki/Accelerated_failure_time_model) model which is a parametric survival regression model for censored data. -It describes a model for the log of survival time, so it's often called -log-linear model for survival analysis. Different from +It describes a model for the log of survival time, so it's often called a +log-linear model for survival analysis. Different from a [Proportional hazards](https://en.wikipedia.org/wiki/Proportional_hazards_model) model -designed for the same purpose, the AFT model is more easily to parallelize -because each instance contribute to the objective function independently. +designed for the same purpose, the AFT model is easier to parallelize +because each instance contributes to the objective function independently. Given the values of the covariates $x^{'}$, for random lifetime $t_{i}$ of subjects i = 1, ..., n, with possible right-censoring, @@ -494,10 +501,10 @@ assumes the form: \iota(\beta,\sigma)=\sum_{i=1}^{n}[-\delta_{i}\log\sigma+\delta_{i}\log{f_{0}}(\epsilon_{i})+(1-\delta_{i})\log{S_{0}(\epsilon_{i})}] \]` Where $S_{0}(\epsilon_{i})$ is the baseline survivor function, -and $f_{0}(\epsilon_{i})$ is corresponding density function. +and $f_{0}(\epsilon_{i})$ is the corresponding density function. The most commonly used AFT model is based on the Weibull distribution of the survival time. -The Weibull distribution for lifetime corresponding to extreme value distribution for +The Weibull distribution for lifetime corresponds to the extreme value distribution for the log of the lifetime, and the $S_{0}(\epsilon)$ function is: `\[ S_{0}(\epsilon_{i})=\exp(-e^{\epsilon_{i}}) @@ -506,7 +513,7 @@ the $f_{0}(\epsilon_{i})$ function is: `\[ f_{0}(\epsilon_{i})=e^{\epsilon_{i}}\exp(-e^{\epsilon_{i}}) \]` -The log-likelihood function for AFT model with Weibull distribution of lifetime is: +The log-likelihood function for AFT model with a Weibull distribution of lifetime is: `\[ \iota(\beta,\sigma)= -\sum_{i=1}^n[\delta_{i}\log\sigma-\delta_{i}\epsilon_{i}+e^{\epsilon_{i}}] \]` @@ -522,7 +529,7 @@ The gradient functions for $\beta$ and $\log\sigma$ respectively are: The AFT model can be formulated as a convex optimization problem, i.e. the task of finding a minimizer of a convex function $-\iota(\beta,\sigma)$ -that depends coefficients vector $\beta$ and the log of scale parameter $\log\sigma$. +that depends on the coefficients vector $\beta$ and the log of scale parameter $\log\sigma$. The optimization algorithm underlying the implementation is L-BFGS. The implementation matches the result from R's survival function [survreg](https://stat.ethz.ch/R-manual/R-devel/library/survival/html/survreg.html) diff --git a/docs/ml-clustering.md b/docs/ml-clustering.md index 440c455cd077..8656eb4001f4 100644 --- a/docs/ml-clustering.md +++ b/docs/ml-clustering.md @@ -79,13 +79,17 @@ Refer to the [Java API docs](api/java/org/apache/spark/ml/clustering/KMeans.html {% include_example java/org/apache/spark/examples/ml/JavaKMeansExample.java %} - +
    +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering.KMeans) for more details. +{% include_example python/ml/kmeans_example.py %} +
    + ## Latent Dirichlet allocation (LDA) `LDA` is implemented as an `Estimator` that supports both `EMLDAOptimizer` and `OnlineLDAOptimizer`, -and generates a `LDAModel` as the base models. Expert users may cast a `LDAModel` generated by +and generates a `LDAModel` as the base model. Expert users may cast a `LDAModel` generated by `EMLDAOptimizer` to a `DistributedLDAModel` if needed.
    @@ -104,4 +108,125 @@ Refer to the [Java API docs](api/java/org/apache/spark/ml/clustering/LDA.html) f {% include_example java/org/apache/spark/examples/ml/JavaLDAExample.java %}
    - \ No newline at end of file +
    + +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering.LDA) for more details. + +{% include_example python/ml/lda_example.py %} +
    + + +## Bisecting k-means + +Bisecting k-means is a kind of [hierarchical clustering](https://en.wikipedia.org/wiki/Hierarchical_clustering) using a +divisive (or "top-down") approach: all observations start in one cluster, and splits are performed recursively as one +moves down the hierarchy. + +Bisecting K-means can often be much faster than regular K-means, but it will generally produce a different clustering. + +`BisectingKMeans` is implemented as an `Estimator` and generates a `BisectingKMeansModel` as the base model. + +### Example + +
    + +
    +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.clustering.BisectingKMeans) for more details. + +{% include_example scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala %} +
    + +
    +Refer to the [Java API docs](api/java/org/apache/spark/ml/clustering/BisectingKMeans.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java %} +
    + +
    +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering.BisectingKMeans) for more details. + +{% include_example python/ml/bisecting_k_means_example.py %} +
    +
    + +## Gaussian Mixture Model (GMM) + +A [Gaussian Mixture Model](http://en.wikipedia.org/wiki/Mixture_model#Multivariate_Gaussian_mixture_model) +represents a composite distribution whereby points are drawn from one of *k* Gaussian sub-distributions, +each with its own probability. The `spark.ml` implementation uses the +[expectation-maximization](http://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm) +algorithm to induce the maximum-likelihood model given a set of samples. + +`GaussianMixture` is implemented as an `Estimator` and generates a `GaussianMixtureModel` as the base +model. + +### Input Columns + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescription
    featuresColVector"features"Feature vector
    + +### Output Columns + + + + + + + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescription
    predictionColInt"prediction"Predicted cluster center
    probabilityColVector"probability"Probability of each cluster
    + +### Example + +
    + +
    +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.clustering.GaussianMixture) for more details. + +{% include_example scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala %} +
    + +
    +Refer to the [Java API docs](api/java/org/apache/spark/ml/clustering/GaussianMixture.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java %} +
    + +
    +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering.GaussianMixture) for more details. + +{% include_example python/ml/gaussian_mixture_example.py %} +
    +
    diff --git a/docs/ml-collaborative-filtering.md b/docs/ml-collaborative-filtering.md index 4514a358e12f..bd3d527d9a0e 100644 --- a/docs/ml-collaborative-filtering.md +++ b/docs/ml-collaborative-filtering.md @@ -60,7 +60,7 @@ best parameter learned from a sampled subset to the full dataset and expect simi
    -In the following example, we load rating data from the +In the following example, we load ratings data from the [MovieLens dataset](http://grouplens.org/datasets/movielens/), each row consisting of a user, a movie, a rating and a timestamp. We then train an ALS model which assumes, by default, that the ratings are @@ -91,7 +91,7 @@ val als = new ALS()
    -In the following example, we load rating data from the +In the following example, we load ratings data from the [MovieLens dataset](http://grouplens.org/datasets/movielens/), each row consisting of a user, a movie, a rating and a timestamp. We then train an ALS model which assumes, by default, that the ratings are @@ -122,7 +122,7 @@ ALS als = new ALS()
    -In the following example, we load rating data from the +In the following example, we load ratings data from the [MovieLens dataset](http://grouplens.org/datasets/movielens/), each row consisting of a user, a movie, a rating and a timestamp. We then train an ALS model which assumes, by default, that the ratings are diff --git a/docs/ml-features.md b/docs/ml-features.md index 0b8f2d773c2e..3db24a384059 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -18,27 +18,60 @@ This section covers algorithms for working with features, roughly divided into t # Feature Extractors -## TF-IDF (HashingTF and IDF) - -[Term Frequency-Inverse Document Frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) is a common text pre-processing step. In Spark ML, TF-IDF is separate into two parts: TF (+hashing) and IDF. +## TF-IDF + +[Term frequency-inverse document frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) +is a feature vectorization method widely used in text mining to reflect the importance of a term +to a document in the corpus. Denote a term by `$t$`, a document by `$d$`, and the corpus by `$D$`. +Term frequency `$TF(t, d)$` is the number of times that term `$t$` appears in document `$d$`, while +document frequency `$DF(t, D)$` is the number of documents that contains term `$t$`. If we only use +term frequency to measure the importance, it is very easy to over-emphasize terms that appear very +often but carry little information about the document, e.g. "a", "the", and "of". If a term appears +very often across the corpus, it means it doesn't carry special information about a particular document. +Inverse document frequency is a numerical measure of how much information a term provides: +`\[ +IDF(t, D) = \log \frac{|D| + 1}{DF(t, D) + 1}, +\]` +where `$|D|$` is the total number of documents in the corpus. Since logarithm is used, if a term +appears in all documents, its IDF value becomes 0. Note that a smoothing term is applied to avoid +dividing by zero for terms outside the corpus. The TF-IDF measure is simply the product of TF and IDF: +`\[ +TFIDF(t, d, D) = TF(t, d) \cdot IDF(t, D). +\]` +There are several variants on the definition of term frequency and document frequency. +In MLlib, we separate TF and IDF to make them flexible. **TF**: Both `HashingTF` and `CountVectorizer` can be used to generate the term frequency vectors. `HashingTF` is a `Transformer` which takes sets of terms and converts those sets into fixed-length feature vectors. In text processing, a "set of terms" might be a bag of words. -The algorithm combines Term Frequency (TF) counts with the -[hashing trick](http://en.wikipedia.org/wiki/Feature_hashing) for dimensionality reduction. +`HashingTF` utilizes the [hashing trick](http://en.wikipedia.org/wiki/Feature_hashing). +A raw feature is mapped into an index (term) by applying a hash function. Then term frequencies +are calculated based on the mapped indices. This approach avoids the need to compute a global +term-to-index map, which can be expensive for a large corpus, but it suffers from potential hash +collisions, where different raw features may become the same term after hashing. To reduce the +chance of collision, we can increase the target feature dimension, i.e. the number of buckets +of the hash table. Since a simple modulo is used to transform the hash function to a column index, +it is advisable to use a power of two as the feature dimension, otherwise the features will +not be mapped evenly to the columns. The default feature dimension is `$2^{18} = 262,144$`. `CountVectorizer` converts text documents to vectors of term counts. Refer to [CountVectorizer ](ml-features.html#countvectorizer) for more details. **IDF**: `IDF` is an `Estimator` which is fit on a dataset and produces an `IDFModel`. The -`IDFModel` takes feature vectors (generally created from `HashingTF` or `CountVectorizer`) and scales each column. -Intuitively, it down-weights columns which appear frequently in a corpus. +`IDFModel` takes feature vectors (generally created from `HashingTF` or `CountVectorizer`) and +scales each column. Intuitively, it down-weights columns which appear frequently in a corpus. -Please refer to the [MLlib user guide on TF-IDF](mllib-feature-extraction.html#tf-idf) for more details on Term Frequency and Inverse Document Frequency. +**Note:** `spark.ml` doesn't provide tools for text segmentation. +We refer users to the [Stanford NLP Group](http://nlp.stanford.edu/) and +[scalanlp/chalk](https://github.com/scalanlp/chalk). -In the following code segment, we start with a set of sentences. We split each sentence into words using `Tokenizer`. For each sentence (bag of words), we use `HashingTF` to hash the sentence into a feature vector. We use `IDF` to rescale the feature vectors; this generally improves performance when using text as features. Our feature vectors could then be passed to a learning algorithm. +**Examples** + +In the following code segment, we start with a set of sentences. We split each sentence into words +using `Tokenizer`. For each sentence (bag of words), we use `HashingTF` to hash the sentence into +a feature vector. We use `IDF` to rescale the feature vectors; this generally improves performance +when using text as features. Our feature vectors could then be passed to a learning algorithm.
    @@ -71,7 +104,7 @@ the [IDF Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.IDF) for mor `Word2Vec` is an `Estimator` which takes sequences of words representing documents and trains a `Word2VecModel`. The model maps each word to a unique fixed-size vector. The `Word2VecModel` transforms each document into a vector using the average of all words in the document; this vector -can then be used for as features for prediction, document similarity calculations, etc. +can then be used as features for prediction, document similarity calculations, etc. Please refer to the [MLlib user guide on Word2Vec](mllib-feature-extraction.html#word2vec) for more details. @@ -107,12 +140,12 @@ for more details on the API. `CountVectorizer` and `CountVectorizerModel` aim to help convert a collection of text documents to vectors of token counts. When an a-priori dictionary is not available, `CountVectorizer` can - be used as an `Estimator` to extract the vocabulary and generates a `CountVectorizerModel`. The + be used as an `Estimator` to extract the vocabulary, and generates a `CountVectorizerModel`. The model produces sparse representations for the documents over the vocabulary, which can then be passed to other algorithms like LDA. During the fitting process, `CountVectorizer` will select the top `vocabSize` words ordered by - term frequency across the corpus. An optional parameter "minDF" also affect the fitting process + term frequency across the corpus. An optional parameter "minDF" also affects the fitting process by specifying the minimum number (or fraction if < 1.0) of documents a term must appear in to be included in the vocabulary. @@ -127,9 +160,9 @@ Assume that we have the following DataFrame with columns `id` and `texts`: 1 | Array("a", "b", "b", "c", "a") ~~~~ -each row in`texts` is a document of type Array[String]. -Invoking fit of `CountVectorizer` produces a `CountVectorizerModel` with vocabulary (a, b, c), -then the output column "vector" after transformation contains: +each row in `texts` is a document of type Array[String]. +Invoking fit of `CountVectorizer` produces a `CountVectorizerModel` with vocabulary (a, b, c). +Then the output column "vector" after transformation contains: ~~~~ id | texts | vector @@ -138,7 +171,7 @@ then the output column "vector" after transformation contains: 1 | Array("a", "b", "b", "c", "a") | (3,[0,1,2],[2.0,2.0,1.0]) ~~~~ -each vector represents the token counts of the document over the vocabulary. +Each vector represents the token counts of the document over the vocabulary.
    @@ -185,7 +218,7 @@ for more details on the API.
    Refer to the [Tokenizer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.Tokenizer) -and the [RegexTokenizer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.Tokenizer) +and the [RegexTokenizer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.RegexTokenizer) for more details on the API. {% include_example scala/org/apache/spark/examples/ml/TokenizerExample.scala %} @@ -444,8 +477,7 @@ for more details on the API. ## StringIndexer `StringIndexer` encodes a string column of labels to a column of label indices. -The indices are in `[0, numLabels)`, ordered by label frequencies. -So the most frequent label gets index `0`. +The indices are in `[0, numLabels)`, ordered by label frequencies, so the most frequent label gets index `0`. If the input column is numeric, we cast it to string and index the string values. When downstream pipeline components such as `Estimator` or `Transformer` make use of this string-indexed label, you must set the input @@ -552,7 +584,7 @@ for more details on the API. ## IndexToString Symmetrically to `StringIndexer`, `IndexToString` maps a column of label indices -back to a column containing the original labels as strings. The common use case +back to a column containing the original labels as strings. A common use case is to produce indices from labels with `StringIndexer`, train a model with those indices and retrieve the original labels from the column of predicted indices with `IndexToString`. However, you are free to supply your own labels. @@ -619,7 +651,7 @@ for more details on the API. ## OneHotEncoder -[One-hot encoding](http://en.wikipedia.org/wiki/One-hot) maps a column of label indices to a column of binary vectors, with at most a single one-value. This encoding allows algorithms which expect continuous features, such as Logistic Regression, to use categorical features +[One-hot encoding](http://en.wikipedia.org/wiki/One-hot) maps a column of label indices to a column of binary vectors, with at most a single one-value. This encoding allows algorithms which expect continuous features, such as Logistic Regression, to use categorical features.
    @@ -775,7 +807,7 @@ The rescaled value for a feature E is calculated as, \end{equation}` For the case `E_{max} == E_{min}`, `Rescaled(e_i) = 0.5 * (max + min)` -Note that since zero values will probably be transformed to non-zero values, output of the transformer will be DenseVector even for sparse input. +Note that since zero values will probably be transformed to non-zero values, output of the transformer will be `DenseVector` even for sparse input. The following example demonstrates how to load a dataset in libsvm format and then rescale each feature to [0, 1]. @@ -801,6 +833,7 @@ for more details on the API.
    Refer to the [MinMaxScaler Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.MinMaxScaler) +and the [MinMaxScalerModel Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.MinMaxScalerModel) for more details on the API. {% include_example python/ml/min_max_scaler_example.py %} @@ -841,6 +874,7 @@ for more details on the API.
    Refer to the [MaxAbsScaler Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.MaxAbsScaler) +and the [MaxAbsScalerModel Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.MaxAbsScalerModel) for more details on the API. {% include_example python/ml/max_abs_scaler_example.py %} @@ -853,7 +887,7 @@ for more details on the API. * `splits`: Parameter for mapping continuous features into buckets. With n+1 splits, there are n buckets. A bucket defined by splits x,y holds values in the range [x,y) except the last bucket, which also includes y. Splits should be strictly increasing. Values at -inf, inf must be explicitly provided to cover all Double values; Otherwise, values outside the splits specified will be treated as errors. Two examples of `splits` are `Array(Double.NegativeInfinity, 0.0, 1.0, Double.PositiveInfinity)` and `Array(0.0, 1.0, 2.0)`. -Note that if you have no idea of the upper bound and lower bound of the targeted column, you would better add the `Double.NegativeInfinity` and `Double.PositiveInfinity` as the bounds of your splits to prevent a potential out of Bucketizer bounds exception. +Note that if you have no idea of the upper and lower bounds of the targeted column, you should add `Double.NegativeInfinity` and `Double.PositiveInfinity` as the bounds of your splits to prevent a potential out of Bucketizer bounds exception. Note also that the splits that you provided have to be in strictly increasing order, i.e. `s0 < s1 < s2 < ... < sn`. @@ -941,7 +975,7 @@ for more details on the API. Currently we only support SQL syntax like `"SELECT ... FROM __THIS__ ..."` where `"__THIS__"` represents the underlying table of the input dataset. The select clause specifies the fields, constants, and expressions to display in -the output, it can be any select clause that Spark SQL supports. Users can also +the output, and can be any select clause that Spark SQL supports. Users can also use Spark SQL built-in function and UDFs to operate on these selected columns. For example, `SQLTransformer` supports statements like: @@ -1086,7 +1120,7 @@ Assume that we have a DataFrame with the columns `id`, `hour`: ~~~ `hour` is a continuous feature with `Double` type. We want to turn the continuous feature into -categorical one. Given `numBuckets = 3`, we should get the following DataFrame: +a categorical one. Given `numBuckets = 3`, we should get the following DataFrame: ~~~ id | hour | result @@ -1118,6 +1152,15 @@ for more details on the API. {% include_example java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java %}
    + +
    + +Refer to the [QuantileDiscretizer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.QuantileDiscretizer) +for more details on the API. + +{% include_example python/ml/quantile_discretizer_example.py %} +
    +
    # Feature Selectors @@ -1127,19 +1170,19 @@ for more details on the API. `VectorSlicer` is a transformer that takes a feature vector and outputs a new feature vector with a sub-array of the original features. It is useful for extracting features from a vector column. -`VectorSlicer` accepts a vector column with a specified indices, then outputs a new vector column +`VectorSlicer` accepts a vector column with specified indices, then outputs a new vector column whose values are selected via those indices. There are two types of indices, - 1. Integer indices that represents the indices into the vector, `setIndices()`; + 1. Integer indices that represent the indices into the vector, `setIndices()`. - 2. String indices that represents the names of features into the vector, `setNames()`. + 2. String indices that represent the names of features into the vector, `setNames()`. *This requires the vector column to have an `AttributeGroup` since the implementation matches on the name field of an `Attribute`.* Specification by integer and string are both acceptable. Moreover, you can use integer index and string name simultaneously. At least one feature must be selected. Duplicate features are not allowed, so there can be no overlap between selected indices and names. Note that if names of -features are selected, an exception will be threw out when encountering with empty input attributes. +features are selected, an exception will be thrown if empty input attributes are encountered. The output vector will order features with the selected indices first (in the order given), followed by the selected names (in the order given). @@ -1154,8 +1197,8 @@ Suppose that we have a DataFrame with the column `userFeatures`: [0.0, 10.0, 0.5] ~~~ -`userFeatures` is a vector column that contains three user features. Assuming that the first column -of `userFeatures` are all zeros, so we want to remove it and only the last two columns are selected. +`userFeatures` is a vector column that contains three user features. Assume that the first column +of `userFeatures` are all zeros, so we want to remove it and select only the last two columns. The `VectorSlicer` selects the last two elements with `setIndices(1, 2)` then produces a new vector column named `features`: @@ -1165,7 +1208,7 @@ column named `features`: [0.0, 10.0, 0.5] | [10.0, 0.5] ~~~ -Suppose also that we have a potential input attributes for the `userFeatures`, i.e. +Suppose also that we have potential input attributes for the `userFeatures`, i.e. `["f1", "f2", "f3"]`, then we can use `setNames("f2", "f3")` to select them. ~~~ @@ -1293,8 +1336,8 @@ id | features | clicked 9 | [1.0, 0.0, 15.0, 0.1] | 0.0 ~~~ -If we use `ChiSqSelector` with a `numTopFeatures = 1`, then according to our label `clicked` the -last column in our `features` chosen as the most useful feature: +If we use `ChiSqSelector` with `numTopFeatures = 1`, then according to our label `clicked` the +last column in our `features` is chosen as the most useful feature: ~~~ id | features | clicked | selectedFeatures diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 99167873cd02..dae86d84804d 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -47,7 +47,7 @@ mostly inspired by the [scikit-learn](http://scikit-learn.org/) project. E.g., a `DataFrame` could have different columns storing text, feature vectors, true labels, and predictions. * **[`Transformer`](ml-guide.html#transformers)**: A `Transformer` is an algorithm which can transform one `DataFrame` into another `DataFrame`. -E.g., an ML model is a `Transformer` which transforms `DataFrame` with features into a `DataFrame` with predictions. +E.g., an ML model is a `Transformer` which transforms a `DataFrame` with features into a `DataFrame` with predictions. * **[`Estimator`](ml-guide.html#estimators)**: An `Estimator` is an algorithm which can be fit on a `DataFrame` to produce a `Transformer`. E.g., a learning algorithm is an `Estimator` which trains on a `DataFrame` and produces a model. @@ -257,7 +257,7 @@ Currently, `spark.ml` supports model selection using the [`CrossValidator`](api/ The `Evaluator` can be a [`RegressionEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.RegressionEvaluator) for regression problems, a [`BinaryClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.BinaryClassificationEvaluator) -for binary data, or a [`MultiClassClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator) +for binary data, or a [`MulticlassClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator) for multiclass problems. The default metric used to choose the best `ParamMap` can be overridden by the `setMetricName` method in each of these evaluators. @@ -292,13 +292,13 @@ However, it is also a well-established method for choosing parameters which is m ## Example: model selection via train validation split In addition to `CrossValidator` Spark also offers `TrainValidationSplit` for hyper-parameter tuning. -`TrainValidationSplit` only evaluates each combination of parameters once as opposed to k times in - case of `CrossValidator`. It is therefore less expensive, +`TrainValidationSplit` only evaluates each combination of parameters once, as opposed to k times in + the case of `CrossValidator`. It is therefore less expensive, but will not produce as reliable results when the training dataset is not sufficiently large. `TrainValidationSplit` takes an `Estimator`, a set of `ParamMap`s provided in the `estimatorParamMaps` parameter, and an `Evaluator`. -It begins by splitting the dataset into two parts using `trainRatio` parameter +It begins by splitting the dataset into two parts using the `trainRatio` parameter which are used as separate training and test datasets. For example with `$trainRatio=0.75$` (default), `TrainValidationSplit` will generate a training and test dataset pair where 75% of the data is used for training and 25% for validation. Similar to `CrossValidator`, `TrainValidationSplit` also iterates through the set of `ParamMap`s. diff --git a/docs/mllib-data-types.md b/docs/mllib-data-types.md index 5e3ee472a72c..2ffe0f1c2b31 100644 --- a/docs/mllib-data-types.md +++ b/docs/mllib-data-types.md @@ -314,12 +314,12 @@ matrices. Remember, local matrices in MLlib are stored in column-major order. Refer to the [`Matrix` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.linalg.Matrix) and [`Matrices` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.linalg.Matrices) for more details on the API. {% highlight python %} -import org.apache.spark.mllib.linalg.{Matrix, Matrices} +from pyspark.mllib.linalg import Matrix, Matrices -// Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) +# Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) dm2 = Matrices.dense(3, 2, [1, 2, 3, 4, 5, 6]) -// Create a sparse matrix ((9.0, 0.0), (0.0, 8.0), (0.0, 6.0)) +# Create a sparse matrix ((9.0, 0.0), (0.0, 8.0), (0.0, 6.0)) sm = Matrices.sparse(3, 2, [0, 1, 3], [0, 2, 1], [9, 6, 8]) {% endhighlight %}
    diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 7a9728503265..4c027c84ec90 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -10,6 +10,9 @@ displayTitle: Feature Extraction and Transformation - spark.mllib ## TF-IDF +**Note** We recommend using the DataFrame-based API, which is detailed in the [ML user guide on +TF-IDF](ml-features.html#tf-idf). + [Term frequency-inverse document frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) is a feature vectorization method widely used in text mining to reflect the importance of a term to a document in the corpus. Denote a term by `$t$`, a document by `$d$`, and the corpus by `$D$`. diff --git a/docs/monitoring.md b/docs/monitoring.md index 9912cde743a4..78a3470abf40 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -27,11 +27,6 @@ in the UI to persisted storage. ## Viewing After the Fact -Spark's Standalone Mode cluster manager also has its own -[web UI](spark-standalone.html#monitoring-and-logging). If an application has logged events over -the course of its lifetime, then the Standalone master's web UI will automatically re-render the -application's UI after the application has finished. - If Spark is run on Mesos or YARN, it is still possible to construct the UI of an application through Spark's history server, provided that the application's event logs exist. You can start the history server by executing: @@ -162,8 +157,8 @@ The history server can be configured as follows: If enabled, access control checks are made regardless of what the individual application had set for spark.ui.acls.enable when the application was run. The application owner will always have authorization to view their own application and any users specified via - spark.ui.view.acls when the application was run will also have authorization - to view that application. + spark.ui.view.acls and groups specified via spark.ui.view.acls.groups + when the application was run will also have authorization to view that application. If disabled, no access control checks are made. @@ -341,7 +336,7 @@ keep the paths consistent in both modes. # Metrics Spark has a configurable metrics system based on the -[Coda Hale Metrics Library](http://metrics.codahale.com/). +[Dropwizard Metrics Library](http://metrics.dropwizard.io/). This allows users to report Spark metrics to a variety of sinks including HTTP, JMX, and CSV files. The metrics system is configured via a configuration file that Spark expects to be present at `$SPARK_HOME/conf/metrics.properties`. A custom file location can be specified via the diff --git a/docs/programming-guide.md b/docs/programming-guide.md index cf6f1d89147f..d375926a910e 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -328,7 +328,7 @@ Text file RDDs can be created using `SparkContext`'s `textFile` method. This met {% highlight scala %} scala> val distFile = sc.textFile("data.txt") -distFile: RDD[String] = MappedRDD@1d4cee08 +distFile: org.apache.spark.rdd.RDD[String] = data.txt MapPartitionsRDD[10] at textFile at :26 {% endhighlight %} Once created, `distFile` can be acted on by dataset operations. For example, we can add up the sizes of all the lines using the `map` and `reduce` operations as follows: `distFile.map(s => s.length).reduce((a, b) => a + b)`. diff --git a/docs/quick-start.md b/docs/quick-start.md index d481fe0ea6d7..72372a6bc854 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -33,7 +33,7 @@ Spark's primary abstraction is a distributed collection of items called a Resili {% highlight scala %} scala> val textFile = sc.textFile("README.md") -textFile: spark.RDD[String] = spark.MappedRDD@2ee9b6e3 +textFile: org.apache.spark.rdd.RDD[String] = README.md MapPartitionsRDD[1] at textFile at :25 {% endhighlight %} RDDs have _[actions](programming-guide.html#actions)_, which return values, and _[transformations](programming-guide.html#transformations)_, which return pointers to new RDDs. Let's start with a few actions: @@ -50,7 +50,7 @@ Now let's use a transformation. We will use the [`filter`](programming-guide.htm {% highlight scala %} scala> val linesWithSpark = textFile.filter(line => line.contains("Spark")) -linesWithSpark: spark.RDD[String] = spark.FilteredRDD@7dd4af09 +linesWithSpark: org.apache.spark.rdd.RDD[String] = MapPartitionsRDD[2] at filter at :27 {% endhighlight %} We can chain together transformations and actions: @@ -123,7 +123,7 @@ One common data flow pattern is MapReduce, as popularized by Hadoop. Spark can i {% highlight scala %} scala> val wordCounts = textFile.flatMap(line => line.split(" ")).map(word => (word, 1)).reduceByKey((a, b) => a + b) -wordCounts: spark.RDD[(String, Int)] = spark.ShuffledAggregatedRDD@71f027b8 +wordCounts: org.apache.spark.rdd.RDD[(String, Int)] = ShuffledRDD[8] at reduceByKey at :28 {% endhighlight %} Here, we combined the [`flatMap`](programming-guide.html#transformations), [`map`](programming-guide.html#transformations), and [`reduceByKey`](programming-guide.html#transformations) transformations to compute the per-word counts in the file as an RDD of (String, Int) pairs. To collect the word counts in our shell, we can use the [`collect`](programming-guide.html#actions) action: @@ -181,7 +181,7 @@ Spark also supports pulling data sets into a cluster-wide in-memory cache. This {% highlight scala %} scala> linesWithSpark.cache() -res7: spark.RDD[String] = spark.FilteredRDD@17e51082 +res7: linesWithSpark.type = MapPartitionsRDD[2] at filter at :27 scala> linesWithSpark.count() res8: Long = 19 diff --git a/docs/security.md b/docs/security.md index 32c33d285747..d2708a80703e 100644 --- a/docs/security.md +++ b/docs/security.md @@ -16,10 +16,10 @@ and by using [https/SSL](http://en.wikipedia.org/wiki/HTTPS) via the `spark.ui.h ### Authentication -A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view ACLs to make sure they are authorized to view the UI. The configs `spark.acls.enable` and `spark.ui.view.acls` control the behavior of the ACLs. Note that the user who started the application always has view access to the UI. On YARN, the Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. +A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view ACLs to make sure they are authorized to view the UI. The configs `spark.acls.enable`, `spark.ui.view.acls` and `spark.ui.view.acls.groups` control the behavior of the ACLs. Note that the user who started the application always has view access to the UI. On YARN, the Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. -Spark also supports modify ACLs to control who has access to modify a running Spark application. This includes things like killing the application or a task. This is controlled by the configs `spark.acls.enable` and `spark.modify.acls`. Note that if you are authenticating the web UI, in order to use the kill button on the web UI it might be necessary to add the users in the modify acls to the view acls also. On YARN, the modify acls are passed in and control who has modify access via YARN interfaces. -Spark allows for a set of administrators to be specified in the acls who always have view and modify permissions to all the applications. is controlled by the config `spark.admin.acls`. This is useful on a shared cluster where you might have administrators or support staff who help users debug applications. +Spark also supports modify ACLs to control who has access to modify a running Spark application. This includes things like killing the application or a task. This is controlled by the configs `spark.acls.enable`, `spark.modify.acls` and `spark.modify.acls.groups`. Note that if you are authenticating the web UI, in order to use the kill button on the web UI it might be necessary to add the users in the modify acls to the view acls also. On YARN, the modify acls are passed in and control who has modify access via YARN interfaces. +Spark allows for a set of administrators to be specified in the acls who always have view and modify permissions to all the applications. is controlled by the configs `spark.admin.acls` and `spark.admin.acls.groups`. This is useful on a shared cluster where you might have administrators or support staff who help users debug applications. ## Event Logging diff --git a/docs/sparkr.md b/docs/sparkr.md index 760534ae145f..9b5eaa1ec723 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -115,13 +115,13 @@ head(df) SparkR supports operating on a variety of data sources through the `DataFrame` interface. This section describes the general methods for loading and saving data using Data Sources. You can check the Spark SQL programming guide for more [specific options](sql-programming-guide.html#manually-specifying-options) that are available for the built-in data sources. -The general method for creating DataFrames from data sources is `read.df`. This method takes in the `SQLContext`, the path for the file to load and the type of data source. SparkR supports reading JSON and Parquet files natively and through [Spark Packages](http://spark-packages.org/) you can find data source connectors for popular file formats like [CSV](http://spark-packages.org/package/databricks/spark-csv) and [Avro](http://spark-packages.org/package/databricks/spark-avro). These packages can either be added by +The general method for creating DataFrames from data sources is `read.df`. This method takes in the `SQLContext`, the path for the file to load and the type of data source. SparkR supports reading JSON, CSV and Parquet files natively and through [Spark Packages](http://spark-packages.org/) you can find data source connectors for popular file formats like [Avro](http://spark-packages.org/package/databricks/spark-avro). These packages can either be added by specifying `--packages` with `spark-submit` or `sparkR` commands, or if creating context through `init` you can specify the packages with the `packages` argument.
    {% highlight r %} -sc <- sparkR.init(sparkPackages="com.databricks:spark-csv_2.11:1.0.3") +sc <- sparkR.init(sparkPackages="com.databricks:spark-avro_2.11:2.0.1") sqlContext <- sparkRSQL.init(sc) {% endhighlight %}
    diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index a16a6bb1d93e..a9e1f9d5ce46 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -529,7 +529,7 @@ case class Person(name: String, age: Int) // Create an RDD of Person objects and register it as a table. val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)).toDF() -people.registerTempTable("people") +people.createOrReplaceTempView("people") // SQL statements can be run by using the sql methods provided by sqlContext. val teenagers = sqlContext.sql("SELECT name, age FROM people WHERE age >= 13 AND age <= 19") @@ -605,7 +605,7 @@ JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").m // Apply a schema to an RDD of JavaBeans and register it as a table. DataFrame schemaPeople = sqlContext.createDataFrame(people, Person.class); -schemaPeople.registerTempTable("people"); +schemaPeople.createOrReplaceTempView("people"); // SQL can be run over RDDs that have been registered as tables. DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -643,7 +643,7 @@ people = parts.map(lambda p: Row(name=p[0], age=int(p[1]))) # Infer the schema, and register the DataFrame as a table. schemaPeople = sqlContext.createDataFrame(people) -schemaPeople.registerTempTable("people") +schemaPeople.createOrReplaceTempView("people") # SQL can be run over DataFrames that have been registered as a table. teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -703,8 +703,8 @@ val rowRDD = people.map(_.split(",")).map(p => Row(p(0), p(1).trim)) // Apply the schema to the RDD. val peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema) -// Register the DataFrames as a table. -peopleDataFrame.registerTempTable("people") +// Creates a temporary view using the DataFrame. +peopleDataFrame.createOrReplaceTempView("people") // SQL statements can be run by using the sql methods provided by sqlContext. val results = sqlContext.sql("SELECT name FROM people") @@ -771,10 +771,10 @@ JavaRDD rowRDD = people.map( // Apply the schema to the RDD. DataFrame peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema); -// Register the DataFrame as a table. -peopleDataFrame.registerTempTable("people"); +// Creates a temporary view using the DataFrame. +peopleDataFrame.createOrReplaceTempView("people"); -// SQL can be run over RDDs that have been registered as tables. +// SQL can be run over a temporary view created using DataFrames. DataFrame results = sqlContext.sql("SELECT name FROM people"); // The results of SQL queries are DataFrames and support all the normal RDD operations. @@ -824,8 +824,8 @@ schema = StructType(fields) # Apply the schema to the RDD. schemaPeople = sqlContext.createDataFrame(people, schema) -# Register the DataFrame as a table. -schemaPeople.registerTempTable("people") +# Creates a temporary view using the DataFrame +schemaPeople.createOrReplaceTempView("people") # SQL can be run over DataFrames that have been registered as a table. results = sqlContext.sql("SELECT name FROM people") @@ -844,7 +844,7 @@ for name in names.collect(): # Data Sources Spark SQL supports operating on a variety of data sources through the `DataFrame` interface. -A DataFrame can be operated on as normal RDDs and can also be registered as a temporary table. +A DataFrame can be operated on as normal RDDs and can also be used to create a temporary view. Registering a DataFrame as a table allows you to run SQL queries over its data. This section describes the general methods for loading and saving data using the Spark Data Sources and then goes into specific options that are available for the built-in data sources. @@ -1072,8 +1072,8 @@ people.write.parquet("people.parquet") // The result of loading a Parquet file is also a DataFrame. val parquetFile = sqlContext.read.parquet("people.parquet") -//Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerTempTable("parquetFile") +// Parquet files can also be used to create a temporary view and then used in SQL statements. +parquetFile.createOrReplaceTempView("parquetFile") val teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenagers.map(t => "Name: " + t(0)).collect().foreach(println) {% endhighlight %} @@ -1094,8 +1094,8 @@ schemaPeople.write().parquet("people.parquet"); // The result of loading a parquet file is also a DataFrame. DataFrame parquetFile = sqlContext.read().parquet("people.parquet"); -// Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerTempTable("parquetFile"); +// Parquet files can also be used to create a temporary view and then used in SQL statements. +parquetFile.createOrReplaceTempView("parquetFile"); DataFrame teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); List teenagerNames = teenagers.javaRDD().map(new Function() { public String call(Row row) { @@ -1120,8 +1120,8 @@ schemaPeople.write.parquet("people.parquet") # The result of loading a parquet file is also a DataFrame. parquetFile = sqlContext.read.parquet("people.parquet") -# Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerTempTable("parquetFile"); +# Parquet files can also be used to create a temporary view and then used in SQL statements. +parquetFile.createOrReplaceTempView("parquetFile"); teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenNames = teenagers.map(lambda p: "Name: " + p.name) for teenName in teenNames.collect(): @@ -1144,7 +1144,7 @@ write.parquet(schemaPeople, "people.parquet") # The result of loading a parquet file is also a DataFrame. parquetFile <- read.parquet(sqlContext, "people.parquet") -# Parquet files can also be registered as tables and then used in SQL statements. +# Parquet files can also be used to create a temporary view and then used in SQL statements. registerTempTable(parquetFile, "parquetFile") teenagers <- sql(sqlContext, "SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") schema <- structType(structField("name", "string")) @@ -1506,8 +1506,8 @@ people.printSchema() // |-- age: long (nullable = true) // |-- name: string (nullable = true) -// Register this DataFrame as a table. -people.registerTempTable("people") +// Creates a temporary view using the DataFrame +people.createOrReplaceTempView("people") // SQL statements can be run by using the sql methods provided by sqlContext. val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -1544,8 +1544,8 @@ people.printSchema(); // |-- age: long (nullable = true) // |-- name: string (nullable = true) -// Register this DataFrame as a table. -people.registerTempTable("people"); +// Creates a temporary view using the DataFrame +people.createOrReplaceTempView("people"); // SQL statements can be run by using the sql methods provided by sqlContext. DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); @@ -1582,8 +1582,8 @@ people.printSchema() # |-- age: long (nullable = true) # |-- name: string (nullable = true) -# Register this DataFrame as a table. -people.registerTempTable("people") +# Creates a temporary view using the DataFrame. +people.createOrReplaceTempView("people") # SQL statements can be run by using the sql methods provided by `sqlContext`. teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md index 015a2f1fa0bd..0f1e32212eb4 100644 --- a/docs/streaming-kafka-integration.md +++ b/docs/streaming-kafka-integration.md @@ -14,7 +14,7 @@ Next, we discuss how to use this approach in your streaming application. 1. **Linking:** For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). groupId = org.apache.spark - artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}} + artifactId = spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}} version = {{site.SPARK_VERSION_SHORT}} For Python applications, you will have to add this above library and its dependencies when deploying your application. See the *Deploying* subsection below. @@ -64,14 +64,14 @@ Next, we discuss how to use this approach in your streaming application. 3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications. - For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). - For Python applications which lack SBT/Maven project management, `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is, + For Python applications which lack SBT/Maven project management, `spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is, - ./bin/spark-submit --packages org.apache.spark:spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... + ./bin/spark-submit --packages org.apache.spark:spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... - Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-kafka-assembly` from the - [Maven repository](http://search.maven.org/#search|ga|1|a%3A%22spark-streaming-kafka-assembly_{{site.SCALA_BINARY_VERSION}}%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`. + Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-kafka-0-8-assembly` from the + [Maven repository](http://search.maven.org/#search|ga|1|a%3A%22spark-streaming-kafka-0-8-assembly_{{site.SCALA_BINARY_VERSION}}%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`. ## Approach 2: Direct Approach (No Receivers) This new receiver-less "direct" approach has been introduced in Spark 1.3 to ensure stronger end-to-end guarantees. Instead of using receivers to receive data, this approach periodically queries Kafka for the latest offsets in each topic+partition, and accordingly defines the offset ranges to process in each batch. When the jobs to process the data are launched, Kafka's simple consumer API is used to read the defined ranges of offsets from Kafka (similar to read files from a file system). Note that this is an experimental feature introduced in Spark 1.3 for the Scala and Java API, in Spark 1.4 for the Python API. @@ -91,7 +91,7 @@ Next, we discuss how to use this approach in your streaming application. 1. **Linking:** This approach is supported only in Scala/Java application. Link your SBT/Maven project with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). groupId = org.apache.spark - artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}} + artifactId = spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}} version = {{site.SPARK_VERSION_SHORT}} 2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 7f6c0ed6994b..4d0a1122dcd1 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -416,7 +416,7 @@ some of the common ones are as follows. - + @@ -1553,8 +1553,8 @@ words.foreachRDD { rdd => // Convert RDD[String] to DataFrame val wordsDataFrame = rdd.toDF("word") - // Register as table - wordsDataFrame.registerTempTable("words") + // Create a temporary view + wordsDataFrame.createOrReplaceTempView("words") // Do word count on DataFrame using SQL and print it val wordCountsDataFrame = @@ -1606,8 +1606,8 @@ words.foreachRDD( }); DataFrame wordsDataFrame = sqlContext.createDataFrame(rowRDD, JavaRow.class); - // Register as table - wordsDataFrame.registerTempTable("words"); + // Creates a temporary view using the DataFrame + wordsDataFrame.createOrReplaceTempView("words"); // Do word count on table using SQL and print it DataFrame wordCountsDataFrame = @@ -1646,8 +1646,8 @@ def process(time, rdd): rowRdd = rdd.map(lambda w: Row(word=w)) wordsDataFrame = sqlContext.createDataFrame(rowRdd) - # Register as table - wordsDataFrame.registerTempTable("words") + # Creates a temporary view using the DataFrame + wordsDataFrame.createOrReplaceTempView("words") # Do word count on table using SQL and print it wordCountsDataFrame = sqlContext.sql("select word, count(*) as total from words group by word") @@ -1892,7 +1892,7 @@ To run a Spark Streaming applications, you need to have the following. if your application uses [advanced sources](#advanced-sources) (e.g. Kafka, Flume), then you will have to package the extra artifact they link to, along with their dependencies, in the JAR that is used to deploy the application. For example, an application using `KafkaUtils` - will have to include `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and all its + will have to include `spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}` and all its transitive dependencies in the application JAR. - *Configuring sufficient memory for the executors* - Since the received data must be stored in diff --git a/examples/pom.xml b/examples/pom.xml index 4423d0fbe1df..771da5b9a6e6 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -87,7 +87,7 @@ org.apache.spark - spark-streaming-kafka_${scala.binary.version} + spark-streaming-kafka-0-8_${scala.binary.version} ${project.version} provided diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index 31a79ddd3fff..f64155ce3c0c 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -32,8 +32,7 @@ * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ public final class JavaHdfsLR { @@ -43,8 +42,7 @@ public final class JavaHdfsLR { static void showWarning() { String warning = "WARN: This is a naive implementation of Logistic Regression " + "and is given as an example!\n" + - "Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD " + - "or org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS " + + "Please use org.apache.spark.ml.classification.LogisticRegression " + "for more conventional use."; System.err.println(warning); } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java index 22b93a3a85c5..b0115756cf45 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java @@ -21,23 +21,29 @@ import java.util.Arrays; import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.regression.AFTSurvivalRegression; import org.apache.spark.ml.regression.AFTSurvivalRegressionModel; import org.apache.spark.mllib.linalg.*; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; // $example off$ +/** + * An example demonstrating AFTSurvivalRegression. + * Run with + *
    + * bin/run-example ml.JavaAFTSurvivalRegressionExample
    + * 
    + */ public class JavaAFTSurvivalRegressionExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaAFTSurvivalRegressionExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaAFTSurvivalRegressionExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( @@ -52,7 +58,7 @@ public static void main(String[] args) { new StructField("censor", DataTypes.DoubleType, false, Metadata.empty()), new StructField("features", new VectorUDT(), false, Metadata.empty()) }); - Dataset training = jsql.createDataFrame(data, schema); + Dataset training = spark.createDataFrame(data, schema); double[] quantileProbabilities = new double[]{0.3, 0.6}; AFTSurvivalRegression aft = new AFTSurvivalRegression() .setQuantileProbabilities(quantileProbabilities) @@ -66,6 +72,6 @@ public static void main(String[] args) { model.transform(training).show(false); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java index 088037d427f5..7f568f4e0db4 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java @@ -17,11 +17,9 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.io.Serializable; @@ -31,7 +29,6 @@ import org.apache.spark.ml.evaluation.RegressionEvaluator; import org.apache.spark.ml.recommendation.ALS; import org.apache.spark.ml.recommendation.ALSModel; -import org.apache.spark.sql.types.DataTypes; // $example off$ public class JavaALSExample { @@ -83,18 +80,20 @@ public static Rating parseRating(String str) { // $example off$ public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaALSExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaALSExample") + .getOrCreate(); // $example on$ - JavaRDD ratingsRDD = jsc.textFile("data/mllib/als/sample_movielens_ratings.txt") + JavaRDD ratingsRDD = spark + .read().text("data/mllib/als/sample_movielens_ratings.txt").javaRDD() .map(new Function() { public Rating call(String str) { return Rating.parseRating(str); } }); - Dataset ratings = sqlContext.createDataFrame(ratingsRDD, Rating.class); + Dataset ratings = spark.createDataFrame(ratingsRDD, Rating.class); Dataset[] splits = ratings.randomSplit(new double[]{0.8, 0.2}); Dataset training = splits[0]; Dataset test = splits[1]; @@ -109,10 +108,7 @@ public Rating call(String str) { ALSModel model = als.fit(training); // Evaluate the model by computing the RMSE on the test data - Dataset rawPredictions = model.transform(test); - Dataset predictions = rawPredictions - .withColumn("rating", rawPredictions.col("rating").cast(DataTypes.DoubleType)) - .withColumn("prediction", rawPredictions.col("prediction").cast(DataTypes.DoubleType)); + Dataset predictions = model.transform(test); RegressionEvaluator evaluator = new RegressionEvaluator() .setMetricName("rmse") @@ -121,6 +117,6 @@ public Rating call(String str) { Double rmse = evaluator.evaluate(predictions); System.out.println("Root-mean-square error = " + rmse); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java index 0a6e9c2a1f93..5f964aca9209 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java @@ -17,15 +17,13 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.Binarizer; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; @@ -37,21 +35,22 @@ public class JavaBinarizerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaBinarizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaBinarizerExample") + .getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, 0.1), RowFactory.create(1, 0.8), RowFactory.create(2, 0.2) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), new StructField("feature", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset continuousDataFrame = jsql.createDataFrame(jrdd, schema); + Dataset continuousDataFrame = spark.createDataFrame(data, schema); Binarizer binarizer = new Binarizer() .setInputCol("feature") .setOutputCol("binarized_feature") @@ -63,6 +62,6 @@ public static void main(String[] args) { System.out.println(binarized_value); } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java index 1d1a518bbca1..8c82aaaacca3 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java @@ -17,65 +17,51 @@ package org.apache.spark.examples.ml; -import java.util.Arrays; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; // $example on$ import org.apache.spark.ml.clustering.BisectingKMeans; import org.apache.spark.ml.clustering.BisectingKMeansModel; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; // $example off$ +import org.apache.spark.sql.SparkSession; /** - * An example demonstrating a bisecting k-means clustering. + * An example demonstrating bisecting k-means clustering. + * Run with + *
    + * bin/run-example ml.JavaBisectingKMeansExample
    + * 
    */ public class JavaBisectingKMeansExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaBisectingKMeansExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaBisectingKMeansExample") + .getOrCreate(); // $example on$ - JavaRDD data = jsc.parallelize(Arrays.asList( - RowFactory.create(Vectors.dense(0.1, 0.1, 0.1)), - RowFactory.create(Vectors.dense(0.3, 0.3, 0.25)), - RowFactory.create(Vectors.dense(0.1, 0.1, -0.1)), - RowFactory.create(Vectors.dense(20.3, 20.1, 19.9)), - RowFactory.create(Vectors.dense(20.2, 20.1, 19.7)), - RowFactory.create(Vectors.dense(18.9, 20.0, 19.7)) - )); - - StructType schema = new StructType(new StructField[]{ - new StructField("features", new VectorUDT(), false, Metadata.empty()), - }); - - Dataset dataset = jsql.createDataFrame(data, schema); + // Loads data. + Dataset dataset = spark.read().format("libsvm").load("data/mllib/sample_kmeans_data.txt"); - BisectingKMeans bkm = new BisectingKMeans().setK(2); + // Trains a bisecting k-means model. + BisectingKMeans bkm = new BisectingKMeans().setK(2).setSeed(1); BisectingKMeansModel model = bkm.fit(dataset); - System.out.println("Compute Cost: " + model.computeCost(dataset)); + // Evaluate clustering. + double cost = model.computeCost(dataset); + System.out.println("Within Set Sum of Squared Errors = " + cost); - Vector[] clusterCenters = model.clusterCenters(); - for (int i = 0; i < clusterCenters.length; i++) { - Vector clusterCenter = clusterCenters[i]; - System.out.println("Cluster Center " + i + ": " + clusterCenter); + // Shows the result. + System.out.println("Cluster Centers: "); + Vector[] centers = model.clusterCenters(); + for (Vector center : centers) { + System.out.println(center); } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java index 68ffa702ea5e..691df3887a9b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.Bucketizer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -37,23 +35,24 @@ public class JavaBucketizerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaBucketizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaBucketizerExample") + .getOrCreate(); // $example on$ double[] splits = {Double.NEGATIVE_INFINITY, -0.5, 0.0, 0.5, Double.POSITIVE_INFINITY}; - JavaRDD data = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(-0.5), RowFactory.create(-0.3), RowFactory.create(0.0), RowFactory.create(0.2) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("features", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset dataFrame = jsql.createDataFrame(data, schema); + Dataset dataFrame = spark.createDataFrame(data, schema); Bucketizer bucketizer = new Bucketizer() .setInputCol("features") @@ -64,7 +63,7 @@ public static void main(String[] args) { Dataset bucketedData = bucketizer.transform(dataFrame); bucketedData.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java index b1bf1cfeb215..f8f2fb14be1f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; import org.apache.spark.ml.feature.ChiSqSelector; import org.apache.spark.mllib.linalg.VectorUDT; @@ -39,23 +37,24 @@ public class JavaChiSqSelectorExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaChiSqSelectorExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaChiSqSelectorExample") + .getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(7, Vectors.dense(0.0, 0.0, 18.0, 1.0), 1.0), RowFactory.create(8, Vectors.dense(0.0, 1.0, 12.0, 0.0), 0.0), RowFactory.create(9, Vectors.dense(1.0, 0.0, 15.0, 0.1), 0.0) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), new StructField("features", new VectorUDT(), false, Metadata.empty()), new StructField("clicked", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); ChiSqSelector selector = new ChiSqSelector() .setNumTopFeatures(1) @@ -66,6 +65,6 @@ public static void main(String[] args) { Dataset result = selector.fit(df).transform(df); result.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java index ec3ac202bea4..0a6b13601425 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java @@ -19,36 +19,34 @@ // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.CountVectorizer; import org.apache.spark.ml.feature.CountVectorizerModel; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; // $example off$ public class JavaCountVectorizerExample { public static void main(String[] args) { - - SparkConf conf = new SparkConf().setAppName("JavaCountVectorizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaCountVectorizerExample") + .getOrCreate(); // $example on$ // Input data: Each row is a bag of words from a sentence or document. - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Arrays.asList("a", "b", "c")), RowFactory.create(Arrays.asList("a", "b", "b", "c", "a")) - )); + ); StructType schema = new StructType(new StructField [] { new StructField("text", new ArrayType(DataTypes.StringType, true), false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); // fit a CountVectorizerModel from the corpus CountVectorizerModel cvModel = new CountVectorizer() @@ -66,6 +64,6 @@ public static void main(String[] args) { cvModel.transform(df).show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java index 4b15fde9c35f..eee92c77a8c5 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java @@ -17,15 +17,13 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.DCT; import org.apache.spark.mllib.linalg.VectorUDT; import org.apache.spark.mllib.linalg.Vectors; @@ -38,20 +36,21 @@ public class JavaDCTExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaDCTExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaDCTExample") + .getOrCreate(); // $example on$ - JavaRDD data = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Vectors.dense(0.0, 1.0, -2.0, 3.0)), RowFactory.create(Vectors.dense(-1.0, 2.0, 4.0, -7.0)), RowFactory.create(Vectors.dense(14.0, -2.0, -5.0, 1.0)) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("features", new VectorUDT(), false, Metadata.empty()), }); - Dataset df = jsql.createDataFrame(data, schema); + Dataset df = spark.createDataFrame(data, schema); DCT dct = new DCT() .setInputCol("features") .setOutputCol("featuresDCT") @@ -59,7 +58,7 @@ public static void main(String[] args) { Dataset dctDf = dct.transform(df); dctDf.select("featuresDCT").show(3); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java index 8214952f8069..bdb76f004fd6 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java @@ -17,8 +17,6 @@ // scalastyle:off println package org.apache.spark.examples.ml; // $example on$ -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; import org.apache.spark.ml.PipelineStage; @@ -28,18 +26,19 @@ import org.apache.spark.ml.feature.*; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaDecisionTreeClassificationExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaDecisionTreeClassificationExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaDecisionTreeClassificationExample") + .getOrCreate(); // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - Dataset data = sqlContext + Dataset data = spark .read() .format("libsvm") .load("data/mllib/sample_libsvm_data.txt"); @@ -55,10 +54,10 @@ public static void main(String[] args) { VectorIndexerModel featureIndexer = new VectorIndexer() .setInputCol("features") .setOutputCol("indexedFeatures") - .setMaxCategories(4) // features with > 4 distinct values are treated as continuous + .setMaxCategories(4) // features with > 4 distinct values are treated as continuous. .fit(data); - // Split the data into training and test sets (30% held out for testing) + // Split the data into training and test sets (30% held out for testing). Dataset[] splits = data.randomSplit(new double[]{0.7, 0.3}); Dataset trainingData = splits[0]; Dataset testData = splits[1]; @@ -74,11 +73,11 @@ public static void main(String[] args) { .setOutputCol("predictedLabel") .setLabels(labelIndexer.labels()); - // Chain indexers and tree in a Pipeline + // Chain indexers and tree in a Pipeline. Pipeline pipeline = new Pipeline() .setStages(new PipelineStage[]{labelIndexer, featureIndexer, dt, labelConverter}); - // Train model. This also runs the indexers. + // Train model. This also runs the indexers. PipelineModel model = pipeline.fit(trainingData); // Make predictions. @@ -87,7 +86,7 @@ public static void main(String[] args) { // Select example rows to display. predictions.select("predictedLabel", "label", "features").show(5); - // Select (prediction, true label) and compute test error + // Select (prediction, true label) and compute test error. MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() .setLabelCol("indexedLabel") .setPredictionCol("prediction") @@ -100,6 +99,6 @@ public static void main(String[] args) { System.out.println("Learned classification tree model:\n" + treeModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java index a4f3e97bf318..cffb7139edcc 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java @@ -17,8 +17,6 @@ // scalastyle:off println package org.apache.spark.examples.ml; // $example on$ -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; import org.apache.spark.ml.PipelineStage; @@ -29,17 +27,18 @@ import org.apache.spark.ml.regression.DecisionTreeRegressor; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaDecisionTreeRegressionExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaDecisionTreeRegressionExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaDecisionTreeRegressionExample") + .getOrCreate(); // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - Dataset data = sqlContext.read().format("libsvm") + Dataset data = spark.read().format("libsvm") .load("data/mllib/sample_libsvm_data.txt"); // Automatically identify categorical features, and index them. @@ -50,7 +49,7 @@ public static void main(String[] args) { .setMaxCategories(4) .fit(data); - // Split the data into training and test sets (30% held out for testing) + // Split the data into training and test sets (30% held out for testing). Dataset[] splits = data.randomSplit(new double[]{0.7, 0.3}); Dataset trainingData = splits[0]; Dataset testData = splits[1]; @@ -59,11 +58,11 @@ public static void main(String[] args) { DecisionTreeRegressor dt = new DecisionTreeRegressor() .setFeaturesCol("indexedFeatures"); - // Chain indexer and tree in a Pipeline + // Chain indexer and tree in a Pipeline. Pipeline pipeline = new Pipeline() .setStages(new PipelineStage[]{featureIndexer, dt}); - // Train model. This also runs the indexer. + // Train model. This also runs the indexer. PipelineModel model = pipeline.fit(trainingData); // Make predictions. @@ -72,7 +71,7 @@ public static void main(String[] args) { // Select example rows to display. predictions.select("label", "features").show(5); - // Select (prediction, true label) and compute test error + // Select (prediction, true label) and compute test error. RegressionEvaluator evaluator = new RegressionEvaluator() .setLabelCol("label") .setPredictionCol("prediction") @@ -85,6 +84,6 @@ public static void main(String[] args) { System.out.println("Learned regression tree model:\n" + treeModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java index 0ba94786d4e5..8971c0ac002b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java @@ -21,21 +21,19 @@ import com.google.common.collect.Lists; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.Classifier; import org.apache.spark.ml.classification.ClassificationModel; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.BLAS; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.ml.param.IntParam; import org.apache.spark.ml.param.ParamMap; import org.apache.spark.ml.util.Identifiable$; -import org.apache.spark.mllib.linalg.BLAS; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** @@ -51,9 +49,10 @@ public class JavaDeveloperApiExample { public static void main(String[] args) throws Exception { - SparkConf conf = new SparkConf().setAppName("JavaDeveloperApiExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaDeveloperApiExample") + .getOrCreate(); // Prepare training data. List localTraining = Lists.newArrayList( @@ -61,10 +60,9 @@ public static void main(String[] args) throws Exception { new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); - Dataset training = jsql.createDataFrame( - jsc.parallelize(localTraining), LabeledPoint.class); + Dataset training = spark.createDataFrame(localTraining, LabeledPoint.class); - // Create a LogisticRegression instance. This instance is an Estimator. + // Create a LogisticRegression instance. This instance is an Estimator. MyJavaLogisticRegression lr = new MyJavaLogisticRegression(); // Print out the parameters, documentation, and any default values. System.out.println("MyJavaLogisticRegression parameters:\n" + lr.explainParams() + "\n"); @@ -72,7 +70,7 @@ public static void main(String[] args) throws Exception { // We may set parameters using setter methods. lr.setMaxIter(10); - // Learn a LogisticRegression model. This uses the parameters stored in lr. + // Learn a LogisticRegression model. This uses the parameters stored in lr. MyJavaLogisticRegressionModel model = lr.fit(training); // Prepare test data. @@ -80,7 +78,7 @@ public static void main(String[] args) throws Exception { new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); - Dataset test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class); + Dataset test = spark.createDataFrame(localTest, LabeledPoint.class); // Make predictions on test documents. cvModel uses the best model found (lrModel). Dataset results = model.transform(test); @@ -93,7 +91,7 @@ public static void main(String[] args) throws Exception { " even though all coefficients are 0!"); } - jsc.stop(); + spark.stop(); } } @@ -216,7 +214,7 @@ public Vector predictRaw(Vector features) { } /** - * Number of classes the label can take. 2 indicates binary classification. + * Number of classes the label can take. 2 indicates binary classification. */ public int numClasses() { return 2; } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java index 37de9cf3596a..d2e70c23babc 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java @@ -17,21 +17,18 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.ElementwiseProduct; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.types.DataTypes; @@ -41,16 +38,17 @@ public class JavaElementwiseProductExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaElementwiseProductExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaElementwiseProductExample") + .getOrCreate(); // $example on$ // Create some vector data; also works for sparse vectors - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create("a", Vectors.dense(1.0, 2.0, 3.0)), RowFactory.create("b", Vectors.dense(4.0, 5.0, 6.0)) - )); + ); List fields = new ArrayList<>(2); fields.add(DataTypes.createStructField("id", DataTypes.StringType, false)); @@ -58,7 +56,7 @@ public static void main(String[] args) { StructType schema = DataTypes.createStructType(fields); - Dataset dataFrame = sqlContext.createDataFrame(jrdd, schema); + Dataset dataFrame = spark.createDataFrame(data, schema); Vector transformingVector = Vectors.dense(0.0, 1.0, 2.0); @@ -70,6 +68,6 @@ public static void main(String[] args) { // Batch transform the vectors to create new column: transformer.transform(dataFrame).show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java index 604b193dd489..889f5785dfd8 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java @@ -21,8 +21,6 @@ import java.util.Arrays; // $example off$ -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; // $example on$ import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.ml.classification.LogisticRegressionModel; @@ -32,23 +30,23 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * Java example for Estimator, Transformer, and Param. */ public class JavaEstimatorTransformerParamExample { public static void main(String[] args) { - SparkConf conf = new SparkConf() - .setAppName("JavaEstimatorTransformerParamExample"); - SparkContext sc = new SparkContext(conf); - SQLContext sqlContext = new SQLContext(sc); + SparkSession spark = SparkSession + .builder() + .appName("JavaEstimatorTransformerParamExample") + .getOrCreate(); // $example on$ // Prepare training data. // We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans into // DataFrames, where it uses the bean metadata to infer the schema. - Dataset training = sqlContext.createDataFrame( + Dataset training = spark.createDataFrame( Arrays.asList( new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), @@ -89,7 +87,7 @@ public static void main(String[] args) { System.out.println("Model 2 was fit using parameters: " + model2.parent().extractParamMap()); // Prepare test documents. - Dataset test = sqlContext.createDataFrame(Arrays.asList( + Dataset test = spark.createDataFrame(Arrays.asList( new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)) @@ -107,6 +105,6 @@ public static void main(String[] args) { } // $example off$ - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java new file mode 100644 index 000000000000..526bed93fbd2 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java @@ -0,0 +1,64 @@ +/* + * 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.examples.ml; + +// $example on$ +import org.apache.spark.ml.clustering.GaussianMixture; +import org.apache.spark.ml.clustering.GaussianMixtureModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ +import org.apache.spark.sql.SparkSession; + + +/** + * An example demonstrating Gaussian Mixture Model. + * Run with + *
    + * bin/run-example ml.JavaGaussianMixtureExample
    + * 
    + */ +public class JavaGaussianMixtureExample { + + public static void main(String[] args) { + + // Creates a SparkSession + SparkSession spark = SparkSession + .builder() + .appName("JavaGaussianMixtureExample") + .getOrCreate(); + + // $example on$ + // Loads data + Dataset dataset = spark.read().format("libsvm").load("data/mllib/sample_kmeans_data.txt"); + + // Trains a GaussianMixture model + GaussianMixture gmm = new GaussianMixture() + .setK(2); + GaussianMixtureModel model = gmm.fit(dataset); + + // Output the parameters of the mixture model + for (int i = 0; i < model.getK(); i++) { + System.out.printf("weight=%f\nmu=%s\nsigma=\n%s\n", + model.weights()[i], model.gaussians()[i].mean(), model.gaussians()[i].cov()); + } + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java new file mode 100644 index 000000000000..3f072d1e50eb --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java @@ -0,0 +1,83 @@ +/* + * 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.examples.ml; + +// $example on$ +import java.util.Arrays; + +import org.apache.spark.ml.regression.GeneralizedLinearRegression; +import org.apache.spark.ml.regression.GeneralizedLinearRegressionModel; +import org.apache.spark.ml.regression.GeneralizedLinearRegressionTrainingSummary; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ +import org.apache.spark.sql.SparkSession; + +/** + * An example demonstrating generalized linear regression. + * Run with + *
    + * bin/run-example ml.JavaGeneralizedLinearRegressionExample
    + * 
    + */ + +public class JavaGeneralizedLinearRegressionExample { + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaGeneralizedLinearRegressionExample") + .getOrCreate(); + + // $example on$ + // Load training data + Dataset dataset = spark.read().format("libsvm") + .load("data/mllib/sample_linear_regression_data.txt"); + + GeneralizedLinearRegression glr = new GeneralizedLinearRegression() + .setFamily("gaussian") + .setLink("identity") + .setMaxIter(10) + .setRegParam(0.3); + + // Fit the model + GeneralizedLinearRegressionModel model = glr.fit(dataset); + + // Print the coefficients and intercept for generalized linear regression model + System.out.println("Coefficients: " + model.coefficients()); + System.out.println("Intercept: " + model.intercept()); + + // Summarize the model over the training set and print out some metrics + GeneralizedLinearRegressionTrainingSummary summary = model.summary(); + System.out.println("Coefficient Standard Errors: " + + Arrays.toString(summary.coefficientStandardErrors())); + System.out.println("T Values: " + Arrays.toString(summary.tValues())); + System.out.println("P Values: " + Arrays.toString(summary.pValues())); + System.out.println("Dispersion: " + summary.dispersion()); + System.out.println("Null Deviance: " + summary.nullDeviance()); + System.out.println("Residual Degree Of Freedom Null: " + summary.residualDegreeOfFreedomNull()); + System.out.println("Deviance: " + summary.deviance()); + System.out.println("Residual Degree Of Freedom: " + summary.residualDegreeOfFreedom()); + System.out.println("AIC: " + summary.aic()); + System.out.println("Deviance Residuals: "); + summary.residuals().show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java index 553070dace88..5c2e03eda90a 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; @@ -29,18 +27,21 @@ import org.apache.spark.ml.feature.*; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaGradientBoostedTreeClassifierExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaGradientBoostedTreeClassifierExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaGradientBoostedTreeClassifierExample") + .getOrCreate(); // $example on$ // Load and parse the data file, converting it to a DataFrame. - Dataset data = sqlContext.read().format("libsvm") + Dataset data = spark + .read() + .format("libsvm") .load("data/mllib/sample_libsvm_data.txt"); // Index labels, adding metadata to the label column. @@ -74,11 +75,11 @@ public static void main(String[] args) { .setOutputCol("predictedLabel") .setLabels(labelIndexer.labels()); - // Chain indexers and GBT in a Pipeline + // Chain indexers and GBT in a Pipeline. Pipeline pipeline = new Pipeline() .setStages(new PipelineStage[] {labelIndexer, featureIndexer, gbt, labelConverter}); - // Train model. This also runs the indexers. + // Train model. This also runs the indexers. PipelineModel model = pipeline.fit(trainingData); // Make predictions. @@ -87,7 +88,7 @@ public static void main(String[] args) { // Select example rows to display. predictions.select("predictedLabel", "label", "features").show(5); - // Select (prediction, true label) and compute test error + // Select (prediction, true label) and compute test error. MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() .setLabelCol("indexedLabel") .setPredictionCol("prediction") @@ -99,6 +100,6 @@ public static void main(String[] args) { System.out.println("Learned classification GBT model:\n" + gbtModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java index 83fd89e3bd59..769b5c3e8525 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; @@ -30,19 +28,19 @@ import org.apache.spark.ml.regression.GBTRegressor; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaGradientBoostedTreeRegressorExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaGradientBoostedTreeRegressorExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaGradientBoostedTreeRegressorExample") + .getOrCreate(); // $example on$ // Load and parse the data file, converting it to a DataFrame. - Dataset data = - sqlContext.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); // Automatically identify categorical features, and index them. // Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -52,7 +50,7 @@ public static void main(String[] args) { .setMaxCategories(4) .fit(data); - // Split the data into training and test sets (30% held out for testing) + // Split the data into training and test sets (30% held out for testing). Dataset[] splits = data.randomSplit(new double[] {0.7, 0.3}); Dataset trainingData = splits[0]; Dataset testData = splits[1]; @@ -63,10 +61,10 @@ public static void main(String[] args) { .setFeaturesCol("indexedFeatures") .setMaxIter(10); - // Chain indexer and GBT in a Pipeline + // Chain indexer and GBT in a Pipeline. Pipeline pipeline = new Pipeline().setStages(new PipelineStage[] {featureIndexer, gbt}); - // Train model. This also runs the indexer. + // Train model. This also runs the indexer. PipelineModel model = pipeline.fit(trainingData); // Make predictions. @@ -75,7 +73,7 @@ public static void main(String[] args) { // Select example rows to display. predictions.select("prediction", "label", "features").show(5); - // Select (prediction, true label) and compute test error + // Select (prediction, true label) and compute test error. RegressionEvaluator evaluator = new RegressionEvaluator() .setLabelCol("label") .setPredictionCol("prediction") @@ -87,6 +85,6 @@ public static void main(String[] args) { System.out.println("Learned regression GBT model:\n" + gbtModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java index 9b8c22f3bdfd..0064beb8c8f3 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; import org.apache.spark.ml.feature.IndexToString; import org.apache.spark.ml.feature.StringIndexer; @@ -39,24 +37,25 @@ public class JavaIndexToStringExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaIndexToStringExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaIndexToStringExample") + .getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, "a"), RowFactory.create(1, "b"), RowFactory.create(2, "c"), RowFactory.create(3, "a"), RowFactory.create(4, "a"), RowFactory.create(5, "c") - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), new StructField("category", DataTypes.StringType, false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); StringIndexerModel indexer = new StringIndexer() .setInputCol("category") @@ -70,6 +69,6 @@ public static void main(String[] args) { Dataset converted = converter.transform(indexed); converted.select("id", "originalCategory").show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java index c5022f4c0b8f..d8f948ae38cb 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java @@ -17,78 +17,45 @@ package org.apache.spark.examples.ml; -import java.util.regex.Pattern; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.catalyst.expressions.GenericRow; // $example on$ import org.apache.spark.ml.clustering.KMeansModel; import org.apache.spark.ml.clustering.KMeans; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; // $example off$ +import org.apache.spark.sql.SparkSession; /** - * An example demonstrating a k-means clustering. + * An example demonstrating k-means clustering. * Run with *
    - * bin/run-example ml.JavaKMeansExample  
    + * bin/run-example ml.JavaKMeansExample
      * 
    */ public class JavaKMeansExample { - private static class ParsePoint implements Function { - private static final Pattern separator = Pattern.compile(" "); - - @Override - public Row call(String line) { - String[] tok = separator.split(line); - double[] point = new double[tok.length]; - for (int i = 0; i < tok.length; ++i) { - point[i] = Double.parseDouble(tok[i]); - } - Vector[] points = {Vectors.dense(point)}; - return new GenericRow(points); - } - } - public static void main(String[] args) { - if (args.length != 2) { - System.err.println("Usage: ml.JavaKMeansExample "); - System.exit(1); - } - String inputFile = args[0]; - int k = Integer.parseInt(args[1]); - - // Parses the arguments - SparkConf conf = new SparkConf().setAppName("JavaKMeansExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + // Create a SparkSession. + SparkSession spark = SparkSession + .builder() + .appName("JavaKMeansExample") + .getOrCreate(); // $example on$ - // Loads data - JavaRDD points = jsc.textFile(inputFile).map(new ParsePoint()); - StructField[] fields = {new StructField("features", new VectorUDT(), false, Metadata.empty())}; - StructType schema = new StructType(fields); - Dataset dataset = sqlContext.createDataFrame(points, schema); + // Loads data. + Dataset dataset = spark.read().format("libsvm").load("data/mllib/sample_kmeans_data.txt"); - // Trains a k-means model - KMeans kmeans = new KMeans() - .setK(k); + // Trains a k-means model. + KMeans kmeans = new KMeans().setK(2).setSeed(1L); KMeansModel model = kmeans.fit(dataset); - // Shows the result + // Evaluate clustering by computing Within Set Sum of Squared Errors. + double WSSSE = model.computeCost(dataset); + System.out.println("Within Set Sum of Squared Errors = " + WSSSE); + + // Shows the result. Vector[] centers = model.clusterCenters(); System.out.println("Cluster Centers: "); for (Vector center: centers) { @@ -96,6 +63,6 @@ public static void main(String[] args) { } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java index 351bc401180c..904124427907 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java @@ -17,28 +17,15 @@ package org.apache.spark.examples.ml; // $example on$ -import java.util.regex.Pattern; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; import org.apache.spark.ml.clustering.LDA; import org.apache.spark.ml.clustering.LDAModel; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.catalyst.expressions.GenericRow; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.SparkSession; // $example off$ /** - * An example demonstrating LDA + * An example demonstrating LDA. * Run with *
      * bin/run-example ml.JavaLDAExample
    @@ -46,52 +33,37 @@
      */
     public class JavaLDAExample {
     
    -  // $example on$
    -  private static class ParseVector implements Function {
    -    private static final Pattern separator = Pattern.compile(" ");
    -
    -    @Override
    -    public Row call(String line) {
    -      String[] tok = separator.split(line);
    -      double[] point = new double[tok.length];
    -      for (int i = 0; i < tok.length; ++i) {
    -        point[i] = Double.parseDouble(tok[i]);
    -      }
    -      Vector[] points = {Vectors.dense(point)};
    -      return new GenericRow(points);
    -    }
    -  }
    -
       public static void main(String[] args) {
    +    // Creates a SparkSession
    +    SparkSession spark = SparkSession
    +      .builder()
    +      .appName("JavaLDAExample")
    +      .getOrCreate();
     
    -    String inputFile = "data/mllib/sample_lda_data.txt";
    +    // $example on$
    +    // Loads data.
    +    Dataset dataset = spark.read().format("libsvm")
    +      .load("data/mllib/sample_lda_libsvm_data.txt");
     
    -    // Parses the arguments
    -    SparkConf conf = new SparkConf().setAppName("JavaLDAExample");
    -    JavaSparkContext jsc = new JavaSparkContext(conf);
    -    SQLContext sqlContext = new SQLContext(jsc);
    -
    -    // Loads data
    -    JavaRDD points = jsc.textFile(inputFile).map(new ParseVector());
    -    StructField[] fields = {new StructField("features", new VectorUDT(), false, Metadata.empty())};
    -    StructType schema = new StructType(fields);
    -    Dataset dataset = sqlContext.createDataFrame(points, schema);
    -
    -    // Trains a LDA model
    -    LDA lda = new LDA()
    -      .setK(10)
    -      .setMaxIter(10);
    +    // Trains a LDA model.
    +    LDA lda = new LDA().setK(10).setMaxIter(10);
         LDAModel model = lda.fit(dataset);
     
    -    System.out.println(model.logLikelihood(dataset));
    -    System.out.println(model.logPerplexity(dataset));
    +    double ll = model.logLikelihood(dataset);
    +    double lp = model.logPerplexity(dataset);
    +    System.out.println("The lower bound on the log likelihood of the entire corpus: " + ll);
    +    System.out.println("The upper bound bound on perplexity: " + lp);
     
    -    // Shows the result
    +    // Describe topics.
         Dataset topics = model.describeTopics(3);
    +    System.out.println("The topics described by their top-weighted terms:");
         topics.show(false);
    -    model.transform(dataset).show(false);
     
    -    jsc.stop();
    +    // Shows the result.
    +    Dataset transformed = model.transform(dataset);
    +    transformed.show(false);
    +    // $example off$
    +
    +    spark.stop();
       }
    -  // $example off$
     }
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java
    index 08fce89359fc..dcd209e28e2b 100644
    --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java
    +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java
    @@ -17,8 +17,6 @@
     
     package org.apache.spark.examples.ml;
     
    -import org.apache.spark.SparkConf;
    -import org.apache.spark.api.java.JavaSparkContext;
     // $example on$
     import org.apache.spark.ml.regression.LinearRegression;
     import org.apache.spark.ml.regression.LinearRegressionModel;
    @@ -26,18 +24,19 @@
     import org.apache.spark.mllib.linalg.Vectors;
     import org.apache.spark.sql.Dataset;
     import org.apache.spark.sql.Row;
    -import org.apache.spark.sql.SQLContext;
    +import org.apache.spark.sql.SparkSession;
     // $example off$
     
     public class JavaLinearRegressionWithElasticNetExample {
       public static void main(String[] args) {
    -    SparkConf conf = new SparkConf().setAppName("JavaLinearRegressionWithElasticNetExample");
    -    JavaSparkContext jsc = new JavaSparkContext(conf);
    -    SQLContext sqlContext = new SQLContext(jsc);
    +    SparkSession spark = SparkSession
    +      .builder()
    +      .appName("JavaLinearRegressionWithElasticNetExample")
    +      .getOrCreate();
     
         // $example on$
    -    // Load training data
    -    Dataset training = sqlContext.read().format("libsvm")
    +    // Load training data.
    +    Dataset training = spark.read().format("libsvm")
           .load("data/mllib/sample_linear_regression_data.txt");
     
         LinearRegression lr = new LinearRegression()
    @@ -45,14 +44,14 @@ public static void main(String[] args) {
           .setRegParam(0.3)
           .setElasticNetParam(0.8);
     
    -    // Fit the model
    +    // Fit the model.
         LinearRegressionModel lrModel = lr.fit(training);
     
    -    // Print the coefficients and intercept for linear regression
    +    // Print the coefficients and intercept for linear regression.
         System.out.println("Coefficients: "
           + lrModel.coefficients() + " Intercept: " + lrModel.intercept());
     
    -    // Summarize the model over the training set and print out some metrics
    +    // Summarize the model over the training set and print out some metrics.
         LinearRegressionTrainingSummary trainingSummary = lrModel.summary();
         System.out.println("numIterations: " + trainingSummary.totalIterations());
         System.out.println("objectiveHistory: " + Vectors.dense(trainingSummary.objectiveHistory()));
    @@ -61,6 +60,6 @@ public static void main(String[] args) {
         System.out.println("r2: " + trainingSummary.r2());
         // $example off$
     
    -    jsc.stop();
    +    spark.stop();
       }
     }
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionSummaryExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionSummaryExample.java
    index 73b028fb4440..dee56799d8ae 100644
    --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionSummaryExample.java
    +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionSummaryExample.java
    @@ -17,8 +17,6 @@
     
     package org.apache.spark.examples.ml;
     
    -import org.apache.spark.SparkConf;
    -import org.apache.spark.api.java.JavaSparkContext;
     // $example on$
     import org.apache.spark.ml.classification.BinaryLogisticRegressionSummary;
     import org.apache.spark.ml.classification.LogisticRegression;
    @@ -26,18 +24,19 @@
     import org.apache.spark.ml.classification.LogisticRegressionTrainingSummary;
     import org.apache.spark.sql.Dataset;
     import org.apache.spark.sql.Row;
    -import org.apache.spark.sql.SQLContext;
    +import org.apache.spark.sql.SparkSession;
     import org.apache.spark.sql.functions;
     // $example off$
     
     public class JavaLogisticRegressionSummaryExample {
       public static void main(String[] args) {
    -    SparkConf conf = new SparkConf().setAppName("JavaLogisticRegressionSummaryExample");
    -    JavaSparkContext jsc = new JavaSparkContext(conf);
    -    SQLContext sqlContext = new SQLContext(jsc);
    +    SparkSession spark = SparkSession
    +      .builder()
    +      .appName("JavaLogisticRegressionSummaryExample")
    +      .getOrCreate();
     
         // Load training data
    -    Dataset training = sqlContext.read().format("libsvm")
    +    Dataset training = spark.read().format("libsvm")
           .load("data/mllib/sample_libsvm_data.txt");
     
         LogisticRegression lr = new LogisticRegression()
    @@ -80,6 +79,6 @@ public static void main(String[] args) {
         lrModel.setThreshold(bestThreshold);
         // $example off$
     
    -    jsc.stop();
    +    spark.stop();
       }
     }
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java
    index 691166852206..6101c79fb0c9 100644
    --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java
    +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java
    @@ -17,25 +17,24 @@
     
     package org.apache.spark.examples.ml;
     
    -import org.apache.spark.SparkConf;
    -import org.apache.spark.api.java.JavaSparkContext;
     // $example on$
     import org.apache.spark.ml.classification.LogisticRegression;
     import org.apache.spark.ml.classification.LogisticRegressionModel;
     import org.apache.spark.sql.Dataset;
     import org.apache.spark.sql.Row;
    -import org.apache.spark.sql.SQLContext;
    +import org.apache.spark.sql.SparkSession;
     // $example off$
     
     public class JavaLogisticRegressionWithElasticNetExample {
       public static void main(String[] args) {
    -    SparkConf conf = new SparkConf().setAppName("JavaLogisticRegressionWithElasticNetExample");
    -    JavaSparkContext jsc = new JavaSparkContext(conf);
    -    SQLContext sqlContext = new SQLContext(jsc);
    +    SparkSession spark = SparkSession
    +      .builder()
    +      .appName("JavaLogisticRegressionWithElasticNetExample")
    +      .getOrCreate();
     
         // $example on$
         // Load training data
    -    Dataset training = sqlContext.read().format("libsvm")
    +    Dataset training = spark.read().format("libsvm")
           .load("data/mllib/sample_libsvm_data.txt");
     
         LogisticRegression lr = new LogisticRegression()
    @@ -51,6 +50,6 @@ public static void main(String[] args) {
           + lrModel.coefficients() + " Intercept: " + lrModel.intercept());
         // $example off$
     
    -    jsc.stop();
    +    spark.stop();
       }
     }
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java
    index a2a072b253f3..9a27b0e9e23b 100644
    --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java
    +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java
    @@ -17,28 +17,30 @@
     
     package org.apache.spark.examples.ml;
     
    -import org.apache.spark.SparkConf;
    -import org.apache.spark.api.java.JavaSparkContext;
     // $example on$
     import org.apache.spark.ml.feature.MaxAbsScaler;
     import org.apache.spark.ml.feature.MaxAbsScalerModel;
     import org.apache.spark.sql.Dataset;
     import org.apache.spark.sql.Row;
     // $example off$
    -import org.apache.spark.sql.SQLContext;
    +import org.apache.spark.sql.SparkSession;
     
     public class JavaMaxAbsScalerExample {
     
       public static void main(String[] args) {
    -    SparkConf conf = new SparkConf().setAppName("JavaMaxAbsScalerExample");
    -    JavaSparkContext jsc = new JavaSparkContext(conf);
    -    SQLContext jsql = new SQLContext(jsc);
    +    SparkSession spark = SparkSession
    +      .builder()
    +      .appName("JavaMaxAbsScalerExample")
    +      .getOrCreate();
     
         // $example on$
    -    Dataset dataFrame = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt");
    +    Dataset dataFrame = spark
    +      .read()
    +      .format("libsvm")
    +      .load("data/mllib/sample_libsvm_data.txt");
         MaxAbsScaler scaler = new MaxAbsScaler()
    -        .setInputCol("features")
    -        .setOutputCol("scaledFeatures");
    +      .setInputCol("features")
    +      .setOutputCol("scaledFeatures");
     
         // Compute summary statistics and generate MaxAbsScalerModel
         MaxAbsScalerModel scalerModel = scaler.fit(dataFrame);
    @@ -47,7 +49,7 @@ public static void main(String[] args) {
         Dataset scaledData = scalerModel.transform(dataFrame);
         scaledData.show();
         // $example off$
    -    jsc.stop();
    +    spark.stop();
       }
     
     }
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java
    index 4aee18eeabfc..37fa1c5434ea 100644
    --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java
    +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java
    @@ -17,9 +17,7 @@
     
     package org.apache.spark.examples.ml;
     
    -import org.apache.spark.SparkConf;
    -import org.apache.spark.api.java.JavaSparkContext;
    -import org.apache.spark.sql.SQLContext;
    +import org.apache.spark.sql.SparkSession;
     
     // $example on$
     import org.apache.spark.ml.feature.MinMaxScaler;
    @@ -30,12 +28,16 @@
     
     public class JavaMinMaxScalerExample {
       public static void main(String[] args) {
    -    SparkConf conf = new SparkConf().setAppName("JaveMinMaxScalerExample");
    -    JavaSparkContext jsc = new JavaSparkContext(conf);
    -    SQLContext jsql = new SQLContext(jsc);
    +    SparkSession spark = SparkSession
    +      .builder()
    +      .appName("JavaMinMaxScalerExample")
    +      .getOrCreate();
     
         // $example on$
    -    Dataset dataFrame = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt");
    +    Dataset dataFrame = spark
    +      .read()
    +      .format("libsvm")
    +      .load("data/mllib/sample_libsvm_data.txt");
         MinMaxScaler scaler = new MinMaxScaler()
           .setInputCol("features")
           .setOutputCol("scaledFeatures");
    @@ -47,6 +49,6 @@ public static void main(String[] args) {
         Dataset scaledData = scalerModel.transform(dataFrame);
         scaledData.show();
         // $example off$
    -    jsc.stop();
    +    spark.stop();
       }
     }
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java
    index c4122d1247a9..975c65edc0ca 100644
    --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java
    +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java
    @@ -21,8 +21,6 @@
     import java.util.Arrays;
     // $example off$
     
    -import org.apache.spark.SparkConf;
    -import org.apache.spark.SparkContext;
     // $example on$
     import org.apache.spark.ml.Pipeline;
     import org.apache.spark.ml.PipelineStage;
    @@ -37,21 +35,21 @@
     import org.apache.spark.sql.Dataset;
     import org.apache.spark.sql.Row;
     // $example off$
    -import org.apache.spark.sql.SQLContext;
    +import org.apache.spark.sql.SparkSession;
     
     /**
      * Java example for Model Selection via Cross Validation.
      */
     public class JavaModelSelectionViaCrossValidationExample {
       public static void main(String[] args) {
    -    SparkConf conf = new SparkConf()
    -      .setAppName("JavaModelSelectionViaCrossValidationExample");
    -    SparkContext sc = new SparkContext(conf);
    -    SQLContext sqlContext = new SQLContext(sc);
    +    SparkSession spark = SparkSession
    +      .builder()
    +      .appName("JavaModelSelectionViaCrossValidationExample")
    +      .getOrCreate();
     
         // $example on$
         // Prepare training documents, which are labeled.
    -    Dataset training = sqlContext.createDataFrame(Arrays.asList(
    +    Dataset training = spark.createDataFrame(Arrays.asList(
           new JavaLabeledDocument(0L, "a b c d e spark", 1.0),
           new JavaLabeledDocument(1L, "b d", 0.0),
           new JavaLabeledDocument(2L,"spark f g h", 1.0),
    @@ -102,7 +100,7 @@ public static void main(String[] args) {
         CrossValidatorModel cvModel = cv.fit(training);
     
         // Prepare test documents, which are unlabeled.
    -    Dataset test = sqlContext.createDataFrame(Arrays.asList(
    +    Dataset test = spark.createDataFrame(Arrays.asList(
           new JavaDocument(4L, "spark i j k"),
           new JavaDocument(5L, "l m n"),
           new JavaDocument(6L, "mapreduce spark"),
    @@ -117,6 +115,6 @@ public static void main(String[] args) {
         }
         // $example off$
     
    -    sc.stop();
    +    spark.stop();
       }
     }
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java
    index 4994f8f9fa85..0f96293f0348 100644
    --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java
    +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java
    @@ -17,8 +17,6 @@
     
     package org.apache.spark.examples.ml;
     
    -import org.apache.spark.SparkConf;
    -import org.apache.spark.SparkContext;
     // $example on$
     import org.apache.spark.ml.evaluation.RegressionEvaluator;
     import org.apache.spark.ml.param.ParamMap;
    @@ -29,7 +27,7 @@
     import org.apache.spark.sql.Dataset;
     import org.apache.spark.sql.Row;
     // $example off$
    -import org.apache.spark.sql.SQLContext;
    +import org.apache.spark.sql.SparkSession;
     
     /**
      * Java example demonstrating model selection using TrainValidationSplit.
    @@ -44,13 +42,13 @@
      */
     public class JavaModelSelectionViaTrainValidationSplitExample {
       public static void main(String[] args) {
    -    SparkConf conf = new SparkConf()
    -      .setAppName("JavaModelSelectionViaTrainValidationSplitExample");
    -    SparkContext sc = new SparkContext(conf);
    -    SQLContext jsql = new SQLContext(sc);
    +    SparkSession spark = SparkSession
    +      .builder()
    +      .appName("JavaModelSelectionViaTrainValidationSplitExample")
    +      .getOrCreate();
     
         // $example on$
    -    Dataset data = jsql.read().format("libsvm")
    +    Dataset data = spark.read().format("libsvm")
           .load("data/mllib/sample_linear_regression_data.txt");
     
         // Prepare training and test data.
    @@ -87,6 +85,6 @@ public static void main(String[] args) {
           .show();
         // $example off$
     
    -    sc.stop();
    +    spark.stop();
       }
     }
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java
    index 0ca528d8cd07..c7d03d8593a3 100644
    --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java
    +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java
    @@ -18,11 +18,9 @@
     package org.apache.spark.examples.ml;
     
     // $example on$
    -import org.apache.spark.SparkConf;
    -import org.apache.spark.api.java.JavaSparkContext;
     import org.apache.spark.sql.Dataset;
     import org.apache.spark.sql.Row;
    -import org.apache.spark.sql.SQLContext;
    +import org.apache.spark.sql.SparkSession;
     import org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel;
     import org.apache.spark.ml.classification.MultilayerPerceptronClassifier;
     import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator;
    @@ -34,14 +32,15 @@
     public class JavaMultilayerPerceptronClassifierExample {
     
       public static void main(String[] args) {
    -    SparkConf conf = new SparkConf().setAppName("JavaMultilayerPerceptronClassifierExample");
    -    JavaSparkContext jsc = new JavaSparkContext(conf);
    -    SQLContext jsql = new SQLContext(jsc);
    +    SparkSession spark = SparkSession
    +      .builder()
    +      .appName("JavaMultilayerPerceptronClassifierExample")
    +      .getOrCreate();
     
         // $example on$
         // Load training data
         String path = "data/mllib/sample_multiclass_classification_data.txt";
    -    Dataset dataFrame = jsql.read().format("libsvm").load(path);
    +    Dataset dataFrame = spark.read().format("libsvm").load(path);
         // Split the data into train and test
         Dataset[] splits = dataFrame.randomSplit(new double[]{0.6, 0.4}, 1234L);
         Dataset train = splits[0];
    @@ -66,6 +65,6 @@ public static void main(String[] args) {
         System.out.println("Precision = " + evaluator.evaluate(predictionAndLabels));
         // $example off$
     
    -    jsc.stop();
    +    spark.stop();
       }
     }
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java
    index 608bd8028565..899815f57c84 100644
    --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java
    +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java
    @@ -17,15 +17,13 @@
     
     package org.apache.spark.examples.ml;
     
    -import org.apache.spark.SparkConf;
    -import org.apache.spark.api.java.JavaSparkContext;
     import org.apache.spark.sql.Dataset;
    -import org.apache.spark.sql.SQLContext;
    +import org.apache.spark.sql.SparkSession;
     
     // $example on$
     import java.util.Arrays;
    +import java.util.List;
     
    -import org.apache.spark.api.java.JavaRDD;
     import org.apache.spark.ml.feature.NGram;
     import org.apache.spark.sql.Row;
     import org.apache.spark.sql.RowFactory;
    @@ -37,16 +35,17 @@
     
     public class JavaNGramExample {
       public static void main(String[] args) {
    -    SparkConf conf = new SparkConf().setAppName("JavaNGramExample");
    -    JavaSparkContext jsc = new JavaSparkContext(conf);
    -    SQLContext sqlContext = new SQLContext(jsc);
    +    SparkSession spark = SparkSession
    +      .builder()
    +      .appName("JavaNGramExample")
    +      .getOrCreate();
     
         // $example on$
    -    JavaRDD jrdd = jsc.parallelize(Arrays.asList(
    +    List data = Arrays.asList(
           RowFactory.create(0.0, Arrays.asList("Hi", "I", "heard", "about", "Spark")),
           RowFactory.create(1.0, Arrays.asList("I", "wish", "Java", "could", "use", "case", "classes")),
           RowFactory.create(2.0, Arrays.asList("Logistic", "regression", "models", "are", "neat"))
    -    ));
    +    );
     
         StructType schema = new StructType(new StructField[]{
           new StructField("label", DataTypes.DoubleType, false, Metadata.empty()),
    @@ -54,7 +53,7 @@ public static void main(String[] args) {
             "words", DataTypes.createArrayType(DataTypes.StringType), false, Metadata.empty())
         });
     
    -    Dataset wordDataFrame = sqlContext.createDataFrame(jrdd, schema);
    +    Dataset wordDataFrame = spark.createDataFrame(data, schema);
     
         NGram ngramTransformer = new NGram().setInputCol("words").setOutputCol("ngrams");
     
    @@ -66,6 +65,6 @@ public static void main(String[] args) {
           System.out.println();
         }
         // $example off$
    -    jsc.stop();
    +    spark.stop();
       }
     }
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java
    index 41d7ad75b9d4..50a46a5774fc 100644
    --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java
    +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java
    @@ -17,16 +17,13 @@
     
     package org.apache.spark.examples.ml;
     
    -
    -import org.apache.spark.SparkConf;
    -import org.apache.spark.api.java.JavaSparkContext;
     // $example on$
     import org.apache.spark.ml.classification.NaiveBayes;
     import org.apache.spark.ml.classification.NaiveBayesModel;
     import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator;
     import org.apache.spark.sql.Dataset;
     import org.apache.spark.sql.Row;
    -import org.apache.spark.sql.SQLContext;
    +import org.apache.spark.sql.SparkSession;
     // $example off$
     
     /**
    @@ -35,13 +32,15 @@
     public class JavaNaiveBayesExample {
     
       public static void main(String[] args) {
    -    SparkConf conf = new SparkConf().setAppName("JavaNaiveBayesExample");
    -    JavaSparkContext jsc = new JavaSparkContext(conf);
    -    SQLContext jsql = new SQLContext(jsc);
    +    SparkSession spark = SparkSession
    +      .builder()
    +      .appName("JavaNaiveBayesExample")
    +      .getOrCreate();
     
         // $example on$
         // Load training data
    -    Dataset dataFrame = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt");
    +    Dataset dataFrame =
    +      spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt");
         // Split the data into train and test
         Dataset[] splits = dataFrame.randomSplit(new double[]{0.6, 0.4}, 1234L);
         Dataset train = splits[0];
    @@ -59,6 +58,6 @@ public static void main(String[] args) {
         System.out.println("Precision = " + evaluator.evaluate(predictionAndLabels));
         // $example off$
     
    -    jsc.stop();
    +    spark.stop();
       }
     }
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java
    index 31cd75213668..abc38f85ea77 100644
    --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java
    +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java
    @@ -17,9 +17,7 @@
     
     package org.apache.spark.examples.ml;
     
    -import org.apache.spark.SparkConf;
    -import org.apache.spark.api.java.JavaSparkContext;
    -import org.apache.spark.sql.SQLContext;
    +import org.apache.spark.sql.SparkSession;
     
     // $example on$
     import org.apache.spark.ml.feature.Normalizer;
    @@ -29,12 +27,14 @@
     
     public class JavaNormalizerExample {
       public static void main(String[] args) {
    -    SparkConf conf = new SparkConf().setAppName("JavaNormalizerExample");
    -    JavaSparkContext jsc = new JavaSparkContext(conf);
    -    SQLContext jsql = new SQLContext(jsc);
    +    SparkSession spark = SparkSession
    +      .builder()
    +      .appName("JavaNormalizerExample")
    +      .getOrCreate();
     
         // $example on$
    -    Dataset dataFrame = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt");
    +    Dataset dataFrame =
    +      spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt");
     
         // Normalize each Vector using $L^1$ norm.
         Normalizer normalizer = new Normalizer()
    @@ -50,6 +50,6 @@ public static void main(String[] args) {
           normalizer.transform(dataFrame, normalizer.p().w(Double.POSITIVE_INFINITY));
         lInfNormData.show();
         // $example off$
    -    jsc.stop();
    +    spark.stop();
       }
     }
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java
    index 882438ca28eb..5d29e5454921 100644
    --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java
    +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java
    @@ -17,14 +17,12 @@
     
     package org.apache.spark.examples.ml;
     
    -import org.apache.spark.SparkConf;
    -import org.apache.spark.api.java.JavaSparkContext;
    -import org.apache.spark.sql.SQLContext;
    +import org.apache.spark.sql.SparkSession;
     
     // $example on$
     import java.util.Arrays;
    +import java.util.List;
     
    -import org.apache.spark.api.java.JavaRDD;
     import org.apache.spark.ml.feature.OneHotEncoder;
     import org.apache.spark.ml.feature.StringIndexer;
     import org.apache.spark.ml.feature.StringIndexerModel;
    @@ -39,26 +37,27 @@
     
     public class JavaOneHotEncoderExample {
       public static void main(String[] args) {
    -    SparkConf conf = new SparkConf().setAppName("JavaOneHotEncoderExample");
    -    JavaSparkContext jsc = new JavaSparkContext(conf);
    -    SQLContext sqlContext = new SQLContext(jsc);
    +    SparkSession spark = SparkSession
    +      .builder()
    +      .appName("JavaOneHotEncoderExample")
    +      .getOrCreate();
     
         // $example on$
    -    JavaRDD jrdd = jsc.parallelize(Arrays.asList(
    +    List data = Arrays.asList(
           RowFactory.create(0, "a"),
           RowFactory.create(1, "b"),
           RowFactory.create(2, "c"),
           RowFactory.create(3, "a"),
           RowFactory.create(4, "a"),
           RowFactory.create(5, "c")
    -    ));
    +    );
     
         StructType schema = new StructType(new StructField[]{
           new StructField("id", DataTypes.DoubleType, false, Metadata.empty()),
           new StructField("category", DataTypes.StringType, false, Metadata.empty())
         });
     
    -    Dataset df = sqlContext.createDataFrame(jrdd, schema);
    +    Dataset df = spark.createDataFrame(data, schema);
     
         StringIndexerModel indexer = new StringIndexer()
           .setInputCol("category")
    @@ -72,7 +71,7 @@ public static void main(String[] args) {
         Dataset encoded = encoder.transform(indexed);
         encoded.select("id", "categoryVec").show();
         // $example off$
    -    jsc.stop();
    +    spark.stop();
       }
     }
     
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java
    index 1f13b48bf82a..5bf455ebfed2 100644
    --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java
    +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java
    @@ -17,223 +17,68 @@
     
     package org.apache.spark.examples.ml;
     
    -import org.apache.commons.cli.*;
    -
    -import org.apache.spark.SparkConf;
    -import org.apache.spark.api.java.JavaSparkContext;
     // $example on$
     import org.apache.spark.ml.classification.LogisticRegression;
     import org.apache.spark.ml.classification.OneVsRest;
     import org.apache.spark.ml.classification.OneVsRestModel;
    -import org.apache.spark.ml.util.MetadataUtils;
    -import org.apache.spark.mllib.evaluation.MulticlassMetrics;
    -import org.apache.spark.mllib.linalg.Matrix;
    -import org.apache.spark.mllib.linalg.Vector;
    +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator;
     import org.apache.spark.sql.Dataset;
     import org.apache.spark.sql.Row;
    -import org.apache.spark.sql.SQLContext;
    -import org.apache.spark.sql.types.StructField;
     // $example off$
    +import org.apache.spark.sql.SparkSession;
    +
     
     /**
    - * An example runner for Multiclass to Binary Reduction with One Vs Rest.
    - * The example uses Logistic Regression as the base classifier. All parameters that
    - * can be specified on the base classifier can be passed in to the runner options.
    + * An example of Multiclass to Binary Reduction with One Vs Rest,
    + * using Logistic Regression as the base classifier.
      * Run with
      * 
    - * bin/run-example ml.JavaOneVsRestExample [options]
    + * bin/run-example ml.JavaOneVsRestExample
      * 
    */ public class JavaOneVsRestExample { - - private static class Params { - String input; - String testInput = null; - Integer maxIter = 100; - double tol = 1E-6; - boolean fitIntercept = true; - Double regParam = null; - Double elasticNetParam = null; - double fracTest = 0.2; - } - public static void main(String[] args) { - // parse the arguments - Params params = parse(args); - SparkConf conf = new SparkConf().setAppName("JavaOneVsRestExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaOneVsRestExample") + .getOrCreate(); // $example on$ - // configure the base classifier - LogisticRegression classifier = new LogisticRegression() - .setMaxIter(params.maxIter) - .setTol(params.tol) - .setFitIntercept(params.fitIntercept); - - if (params.regParam != null) { - classifier.setRegParam(params.regParam); - } - if (params.elasticNetParam != null) { - classifier.setElasticNetParam(params.elasticNetParam); - } + // load data file. + Dataset inputData = spark.read().format("libsvm") + .load("data/mllib/sample_multiclass_classification_data.txt"); - // instantiate the One Vs Rest Classifier - OneVsRest ovr = new OneVsRest().setClassifier(classifier); + // generate the train/test split. + Dataset[] tmp = inputData.randomSplit(new double[]{0.8, 0.2}); + Dataset train = tmp[0]; + Dataset test = tmp[1]; - String input = params.input; - Dataset inputData = jsql.read().format("libsvm").load(input); - Dataset train; - Dataset test; + // configure the base classifier. + LogisticRegression classifier = new LogisticRegression() + .setMaxIter(10) + .setTol(1E-6) + .setFitIntercept(true); - // compute the train/ test split: if testInput is not provided use part of input - String testInput = params.testInput; - if (testInput != null) { - train = inputData; - // compute the number of features in the training set. - int numFeatures = inputData.first().getAs(1).size(); - test = jsql.read().format("libsvm").option("numFeatures", - String.valueOf(numFeatures)).load(testInput); - } else { - double f = params.fracTest; - Dataset[] tmp = inputData.randomSplit(new double[]{1 - f, f}, 12345); - train = tmp[0]; - test = tmp[1]; - } + // instantiate the One Vs Rest Classifier. + OneVsRest ovr = new OneVsRest().setClassifier(classifier); - // train the multiclass model - OneVsRestModel ovrModel = ovr.fit(train.cache()); + // train the multiclass model. + OneVsRestModel ovrModel = ovr.fit(train); - // score the model on test data - Dataset predictions = ovrModel.transform(test.cache()) + // score the model on test data. + Dataset predictions = ovrModel.transform(test) .select("prediction", "label"); - // obtain metrics - MulticlassMetrics metrics = new MulticlassMetrics(predictions); - StructField predictionColSchema = predictions.schema().apply("prediction"); - Integer numClasses = (Integer) MetadataUtils.getNumClasses(predictionColSchema).get(); + // obtain evaluator. + MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() + .setMetricName("precision"); - // compute the false positive rate per label - StringBuilder results = new StringBuilder(); - results.append("label\tfpr\n"); - for (int label = 0; label < numClasses; label++) { - results.append(label); - results.append("\t"); - results.append(metrics.falsePositiveRate((double) label)); - results.append("\n"); - } - - Matrix confusionMatrix = metrics.confusionMatrix(); - // output the Confusion Matrix - System.out.println("Confusion Matrix"); - System.out.println(confusionMatrix); - System.out.println(); - System.out.println(results); + // compute the classification error on test data. + double precision = evaluator.evaluate(predictions); + System.out.println("Test Error : " + (1 - precision)); // $example off$ - jsc.stop(); - } - - private static Params parse(String[] args) { - Options options = generateCommandlineOptions(); - CommandLineParser parser = new PosixParser(); - Params params = new Params(); - - try { - CommandLine cmd = parser.parse(options, args); - String value; - if (cmd.hasOption("input")) { - params.input = cmd.getOptionValue("input"); - } - if (cmd.hasOption("maxIter")) { - value = cmd.getOptionValue("maxIter"); - params.maxIter = Integer.parseInt(value); - } - if (cmd.hasOption("tol")) { - value = cmd.getOptionValue("tol"); - params.tol = Double.parseDouble(value); - } - if (cmd.hasOption("fitIntercept")) { - value = cmd.getOptionValue("fitIntercept"); - params.fitIntercept = Boolean.parseBoolean(value); - } - if (cmd.hasOption("regParam")) { - value = cmd.getOptionValue("regParam"); - params.regParam = Double.parseDouble(value); - } - if (cmd.hasOption("elasticNetParam")) { - value = cmd.getOptionValue("elasticNetParam"); - params.elasticNetParam = Double.parseDouble(value); - } - if (cmd.hasOption("testInput")) { - value = cmd.getOptionValue("testInput"); - params.testInput = value; - } - if (cmd.hasOption("fracTest")) { - value = cmd.getOptionValue("fracTest"); - params.fracTest = Double.parseDouble(value); - } - - } catch (ParseException e) { - printHelpAndQuit(options); - } - return params; + spark.stop(); } - @SuppressWarnings("static") - private static Options generateCommandlineOptions() { - Option input = OptionBuilder.withArgName("input") - .hasArg() - .isRequired() - .withDescription("input path to labeled examples. This path must be specified") - .create("input"); - Option testInput = OptionBuilder.withArgName("testInput") - .hasArg() - .withDescription("input path to test examples") - .create("testInput"); - Option fracTest = OptionBuilder.withArgName("testInput") - .hasArg() - .withDescription("fraction of data to hold out for testing." + - " If given option testInput, this option is ignored. default: 0.2") - .create("fracTest"); - Option maxIter = OptionBuilder.withArgName("maxIter") - .hasArg() - .withDescription("maximum number of iterations for Logistic Regression. default:100") - .create("maxIter"); - Option tol = OptionBuilder.withArgName("tol") - .hasArg() - .withDescription("the convergence tolerance of iterations " + - "for Logistic Regression. default: 1E-6") - .create("tol"); - Option fitIntercept = OptionBuilder.withArgName("fitIntercept") - .hasArg() - .withDescription("fit intercept for logistic regression. default true") - .create("fitIntercept"); - Option regParam = OptionBuilder.withArgName( "regParam" ) - .hasArg() - .withDescription("the regularization parameter for Logistic Regression.") - .create("regParam"); - Option elasticNetParam = OptionBuilder.withArgName("elasticNetParam" ) - .hasArg() - .withDescription("the ElasticNet mixing parameter for Logistic Regression.") - .create("elasticNetParam"); - - Options options = new Options() - .addOption(input) - .addOption(testInput) - .addOption(fracTest) - .addOption(maxIter) - .addOption(tol) - .addOption(fitIntercept) - .addOption(regParam) - .addOption(elasticNetParam); - - return options; - } - - private static void printHelpAndQuit(Options options) { - HelpFormatter formatter = new HelpFormatter(); - formatter.printHelp("JavaOneVsRestExample", options); - System.exit(-1); - } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java index a792fd7d47cc..ffa979ee013a 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.PCA; import org.apache.spark.ml.feature.PCAModel; import org.apache.spark.mllib.linalg.VectorUDT; @@ -39,22 +37,23 @@ public class JavaPCAExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaPCAExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaPCAExample") + .getOrCreate(); // $example on$ - JavaRDD data = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Vectors.sparse(5, new int[]{1, 3}, new double[]{1.0, 7.0})), RowFactory.create(Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0)), RowFactory.create(Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0)) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("features", new VectorUDT(), false, Metadata.empty()), }); - Dataset df = jsql.createDataFrame(data, schema); + Dataset df = spark.createDataFrame(data, schema); PCAModel pca = new PCA() .setInputCol("features") @@ -65,7 +64,7 @@ public static void main(String[] args) { Dataset result = pca.transform(df).select("pcaFeatures"); result.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java index 305420f208b7..9a43189c9146 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java @@ -19,11 +19,7 @@ // $example on$ import java.util.Arrays; -// $example off$ -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; -// $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; import org.apache.spark.ml.PipelineStage; @@ -33,20 +29,21 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * Java example for simple text document 'Pipeline'. */ public class JavaPipelineExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaPipelineExample"); - SparkContext sc = new SparkContext(conf); - SQLContext sqlContext = new SQLContext(sc); + SparkSession spark = SparkSession + .builder() + .appName("JavaPipelineExample") + .getOrCreate(); // $example on$ // Prepare training documents, which are labeled. - Dataset training = sqlContext.createDataFrame(Arrays.asList( + Dataset training = spark.createDataFrame(Arrays.asList( new JavaLabeledDocument(0L, "a b c d e spark", 1.0), new JavaLabeledDocument(1L, "b d", 0.0), new JavaLabeledDocument(2L, "spark f g h", 1.0), @@ -71,7 +68,7 @@ public static void main(String[] args) { PipelineModel model = pipeline.fit(training); // Prepare test documents, which are unlabeled. - Dataset test = sqlContext.createDataFrame(Arrays.asList( + Dataset test = spark.createDataFrame(Arrays.asList( new JavaDocument(4L, "spark i j k"), new JavaDocument(5L, "l m n"), new JavaDocument(6L, "mapreduce spark"), @@ -86,6 +83,6 @@ public static void main(String[] args) { } // $example off$ - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java index 48fc3c8acb0c..7afcd0e50cd9 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java @@ -17,15 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.PolynomialExpansion; import org.apache.spark.mllib.linalg.VectorUDT; import org.apache.spark.mllib.linalg.Vectors; @@ -39,9 +36,10 @@ public class JavaPolynomialExpansionExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaPolynomialExpansionExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaPolynomialExpansionExample") + .getOrCreate(); // $example on$ PolynomialExpansion polyExpansion = new PolynomialExpansion() @@ -49,17 +47,17 @@ public static void main(String[] args) { .setOutputCol("polyFeatures") .setDegree(3); - JavaRDD data = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Vectors.dense(-2.0, 2.3)), RowFactory.create(Vectors.dense(0.0, 0.0)), RowFactory.create(Vectors.dense(0.6, -1.1)) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("features", new VectorUDT(), false, Metadata.empty()), }); - Dataset df = jsql.createDataFrame(data, schema); + Dataset df = spark.createDataFrame(data, schema); Dataset polyDF = polyExpansion.transform(df); List rows = polyDF.select("polyFeatures").takeAsList(3); @@ -67,6 +65,6 @@ public static void main(String[] args) { System.out.println(r.get(0)); } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java index 7b226fede996..16f58a852d8a 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java @@ -17,13 +17,11 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.QuantileDiscretizer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -36,19 +34,18 @@ public class JavaQuantileDiscretizerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaQuantileDiscretizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaQuantileDiscretizerExample") + .getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize( - Arrays.asList( - RowFactory.create(0, 18.0), - RowFactory.create(1, 19.0), - RowFactory.create(2, 8.0), - RowFactory.create(3, 5.0), - RowFactory.create(4, 2.2) - ) + List data = Arrays.asList( + RowFactory.create(0, 18.0), + RowFactory.create(1, 19.0), + RowFactory.create(2, 8.0), + RowFactory.create(3, 5.0), + RowFactory.create(4, 2.2) ); StructType schema = new StructType(new StructField[]{ @@ -56,7 +53,7 @@ public static void main(String[] args) { new StructField("hour", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); QuantileDiscretizer discretizer = new QuantileDiscretizer() .setInputCol("hour") @@ -66,6 +63,6 @@ public static void main(String[] args) { Dataset result = discretizer.fit(df).transform(df); result.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java index 8c453bf80d64..428067e0f7ef 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.RFormula; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -37,9 +35,10 @@ public class JavaRFormulaExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaRFormulaExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaRFormulaExample") + .getOrCreate(); // $example on$ StructType schema = createStructType(new StructField[]{ @@ -49,13 +48,13 @@ public static void main(String[] args) { createStructField("clicked", DoubleType, false) }); - JavaRDD rdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(7, "US", 18, 1.0), RowFactory.create(8, "CA", 12, 0.0), RowFactory.create(9, "NZ", 15, 0.0) - )); + ); - Dataset dataset = sqlContext.createDataFrame(rdd, schema); + Dataset dataset = spark.createDataFrame(data, schema); RFormula formula = new RFormula() .setFormula("clicked ~ country + hour") .setFeaturesCol("features") @@ -63,7 +62,7 @@ public static void main(String[] args) { Dataset output = formula.fit(dataset).transform(dataset); output.select("features", "label").show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java index 05c2bc9622e1..14af2fbbbbbe 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; @@ -29,19 +27,19 @@ import org.apache.spark.ml.feature.*; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaRandomForestClassifierExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaRandomForestClassifierExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaRandomForestClassifierExample") + .getOrCreate(); // $example on$ // Load and parse the data file, converting it to a DataFrame. - Dataset data = - sqlContext.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); // Index labels, adding metadata to the label column. // Fit on whole dataset to include all labels in index. @@ -98,6 +96,6 @@ public static void main(String[] args) { System.out.println("Learned classification forest model:\n" + rfModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java index d366967083a1..a7078453deb8 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; @@ -30,19 +28,19 @@ import org.apache.spark.ml.regression.RandomForestRegressor; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaRandomForestRegressorExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaRandomForestRegressorExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaRandomForestRegressorExample") + .getOrCreate(); // $example on$ // Load and parse the data file, converting it to a DataFrame. - Dataset data = - sqlContext.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); // Automatically identify categorical features, and index them. // Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -66,7 +64,7 @@ public static void main(String[] args) { Pipeline pipeline = new Pipeline() .setStages(new PipelineStage[] {featureIndexer, rf}); - // Train model. This also runs the indexer. + // Train model. This also runs the indexer. PipelineModel model = pipeline.fit(trainingData); // Make predictions. @@ -87,6 +85,6 @@ public static void main(String[] args) { System.out.println("Learned regression forest model:\n" + rfModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java index 7e3ca99d7cb9..2a3d62de41ab 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java @@ -19,36 +19,34 @@ // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.SQLTransformer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; // $example off$ public class JavaSQLTransformerExample { public static void main(String[] args) { - - SparkConf conf = new SparkConf().setAppName("JavaSQLTransformerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaSQLTransformerExample") + .getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, 1.0, 3.0), RowFactory.create(2, 2.0, 5.0) - )); + ); StructType schema = new StructType(new StructField [] { new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), new StructField("v1", DataTypes.DoubleType, false, Metadata.empty()), new StructField("v2", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); SQLTransformer sqlTrans = new SQLTransformer().setStatement( "SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__"); @@ -56,6 +54,6 @@ public static void main(String[] args) { sqlTrans.transform(df).show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java index cb911ef5ef58..ca80d0d8bba5 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -21,8 +21,6 @@ import com.google.common.collect.Lists; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegressionModel; import org.apache.spark.ml.param.ParamMap; import org.apache.spark.ml.classification.LogisticRegression; @@ -30,7 +28,7 @@ import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * A simple example demonstrating ways to specify parameters for Estimators and Transformers. @@ -42,12 +40,13 @@ public class JavaSimpleParamsExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaSimpleParamsExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaSimpleParamsExample") + .getOrCreate(); // Prepare training data. - // We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans + // We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans // into DataFrames, where it uses the bean metadata to infer the schema. List localTraining = Lists.newArrayList( new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), @@ -55,9 +54,9 @@ public static void main(String[] args) { new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); Dataset training = - jsql.createDataFrame(jsc.parallelize(localTraining), LabeledPoint.class); + spark.createDataFrame(localTraining, LabeledPoint.class); - // Create a LogisticRegression instance. This instance is an Estimator. + // Create a LogisticRegression instance. This instance is an Estimator. LogisticRegression lr = new LogisticRegression(); // Print out the parameters, documentation, and any default values. System.out.println("LogisticRegression parameters:\n" + lr.explainParams() + "\n"); @@ -66,7 +65,7 @@ public static void main(String[] args) { lr.setMaxIter(10) .setRegParam(0.01); - // Learn a LogisticRegression model. This uses the parameters stored in lr. + // Learn a LogisticRegression model. This uses the parameters stored in lr. LogisticRegressionModel model1 = lr.fit(training); // Since model1 is a Model (i.e., a Transformer produced by an Estimator), // we can view the parameters it used during fit(). @@ -78,12 +77,12 @@ public static void main(String[] args) { ParamMap paramMap = new ParamMap(); paramMap.put(lr.maxIter().w(20)); // Specify 1 Param. paramMap.put(lr.maxIter(), 30); // This overwrites the original maxIter. - double[] thresholds = {0.45, 0.55}; + double[] thresholds = {0.5, 0.5}; paramMap.put(lr.regParam().w(0.1), lr.thresholds().w(thresholds)); // Specify multiple Params. // One can also combine ParamMaps. ParamMap paramMap2 = new ParamMap(); - paramMap2.put(lr.probabilityCol().w("myProbability")); // Change output column name + paramMap2.put(lr.probabilityCol().w("myProbability")); // Change output column name. ParamMap paramMapCombined = paramMap.$plus$plus(paramMap2); // Now learn a new model using the paramMapCombined parameters. @@ -96,7 +95,7 @@ public static void main(String[] args) { new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); - Dataset test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class); + Dataset test = spark.createDataFrame(localTest, LabeledPoint.class); // Make predictions on test documents using the Transformer.transform() method. // LogisticRegressionModel.transform will only use the 'features' column. @@ -109,6 +108,6 @@ public static void main(String[] args) { + ", prediction=" + r.get(3)); } - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java index a18a60f44816..7c24c46d2e28 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java @@ -21,8 +21,6 @@ import com.google.common.collect.Lists; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; import org.apache.spark.ml.PipelineStage; @@ -31,7 +29,7 @@ import org.apache.spark.ml.feature.Tokenizer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * A simple text classification pipeline that recognizes "spark" from input text. It uses the Java @@ -44,9 +42,10 @@ public class JavaSimpleTextClassificationPipeline { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaSimpleTextClassificationPipeline"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaSimpleTextClassificationPipeline") + .getOrCreate(); // Prepare training documents, which are labeled. List localTraining = Lists.newArrayList( @@ -55,7 +54,7 @@ public static void main(String[] args) { new LabeledDocument(2L, "spark f g h", 1.0), new LabeledDocument(3L, "hadoop mapreduce", 0.0)); Dataset training = - jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class); + spark.createDataFrame(localTraining, LabeledDocument.class); // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. Tokenizer tokenizer = new Tokenizer() @@ -80,7 +79,7 @@ public static void main(String[] args) { new Document(5L, "l m n"), new Document(6L, "spark hadoop spark"), new Document(7L, "apache hadoop")); - Dataset test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class); + Dataset test = spark.createDataFrame(localTest, Document.class); // Make predictions on test documents. Dataset predictions = model.transform(test); @@ -89,6 +88,6 @@ public static void main(String[] args) { + ", prediction=" + r.get(3)); } - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java index e2dd759c0a40..08ea285a0d53 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java @@ -17,9 +17,7 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import org.apache.spark.ml.feature.StandardScaler; @@ -30,12 +28,14 @@ public class JavaStandardScalerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaStandardScalerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaStandardScalerExample") + .getOrCreate(); // $example on$ - Dataset dataFrame = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset dataFrame = + spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); StandardScaler scaler = new StandardScaler() .setInputCol("features") @@ -50,6 +50,6 @@ public static void main(String[] args) { Dataset scaledData = scalerModel.transform(dataFrame); scaledData.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java index 0ff3782cb3e9..def599442912 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.StopWordsRemover; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -38,28 +36,29 @@ public class JavaStopWordsRemoverExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaStopWordsRemoverExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaStopWordsRemoverExample") + .getOrCreate(); // $example on$ StopWordsRemover remover = new StopWordsRemover() .setInputCol("raw") .setOutputCol("filtered"); - JavaRDD rdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Arrays.asList("I", "saw", "the", "red", "baloon")), RowFactory.create(Arrays.asList("Mary", "had", "a", "little", "lamb")) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField( "raw", DataTypes.createArrayType(DataTypes.StringType), false, Metadata.empty()) }); - Dataset dataset = jsql.createDataFrame(rdd, schema); + Dataset dataset = spark.createDataFrame(data, schema); remover.transform(dataset).show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java index ceacbb4fb3f3..7533c1835e32 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.StringIndexer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -37,30 +35,31 @@ public class JavaStringIndexerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaStringIndexerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaStringIndexerExample") + .getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, "a"), RowFactory.create(1, "b"), RowFactory.create(2, "c"), RowFactory.create(3, "a"), RowFactory.create(4, "a"), RowFactory.create(5, "c") - )); + ); StructType schema = new StructType(new StructField[]{ createStructField("id", IntegerType, false), createStructField("category", StringType, false) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); StringIndexer indexer = new StringIndexer() .setInputCol("category") .setOutputCol("categoryIndex"); Dataset indexed = indexer.fit(df).transform(df); indexed.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java index 107c835f2e01..6e0753959efd 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java @@ -19,10 +19,8 @@ // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.HashingTF; import org.apache.spark.ml.feature.IDF; import org.apache.spark.ml.feature.IDFModel; @@ -31,7 +29,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; @@ -40,21 +38,22 @@ public class JavaTfIdfExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaTfIdfExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaTfIdfExample") + .getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, "Hi I heard about Spark"), RowFactory.create(0, "I wish Java could use case classes"), RowFactory.create(1, "Logistic regression models are neat") - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), new StructField("sentence", DataTypes.StringType, false, Metadata.empty()) }); - Dataset sentenceData = sqlContext.createDataFrame(jrdd, schema); + Dataset sentenceData = spark.createDataFrame(data, schema); Tokenizer tokenizer = new Tokenizer().setInputCol("sentence").setOutputCol("words"); Dataset wordsData = tokenizer.transform(sentenceData); int numFeatures = 20; @@ -76,6 +75,6 @@ public static void main(String[] args) { } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java index 9225fe2262f5..1cc16bb60d17 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.RegexTokenizer; import org.apache.spark.ml.feature.Tokenizer; import org.apache.spark.sql.Dataset; @@ -38,23 +36,24 @@ public class JavaTokenizerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaTokenizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaTokenizerExample") + .getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, "Hi I heard about Spark"), RowFactory.create(1, "I wish Java could use case classes"), RowFactory.create(2, "Logistic,regression,models,are,neat") - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("label", DataTypes.IntegerType, false, Metadata.empty()), new StructField("sentence", DataTypes.StringType, false, Metadata.empty()) }); - Dataset sentenceDataFrame = sqlContext.createDataFrame(jrdd, schema); + Dataset sentenceDataFrame = spark.createDataFrame(data, schema); Tokenizer tokenizer = new Tokenizer().setInputCol("sentence").setOutputCol("words"); @@ -70,6 +69,6 @@ public static void main(String[] args) { .setOutputCol("words") .setPattern("\\W"); // alternatively .setPattern("\\w+").setGaps(false); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java index 953ad455b1dc..41f1d8750ac4 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java @@ -17,14 +17,11 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.VectorAssembler; import org.apache.spark.mllib.linalg.VectorUDT; import org.apache.spark.mllib.linalg.Vectors; @@ -38,9 +35,10 @@ public class JavaVectorAssemblerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaVectorAssemblerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaVectorAssemblerExample") + .getOrCreate(); // $example on$ StructType schema = createStructType(new StructField[]{ @@ -51,8 +49,7 @@ public static void main(String[] args) { createStructField("clicked", DoubleType, false) }); Row row = RowFactory.create(0, 18, 1.0, Vectors.dense(0.0, 10.0, 0.5), 1.0); - JavaRDD rdd = jsc.parallelize(Arrays.asList(row)); - Dataset dataset = sqlContext.createDataFrame(rdd, schema); + Dataset dataset = spark.createDataFrame(Arrays.asList(row), schema); VectorAssembler assembler = new VectorAssembler() .setInputCols(new String[]{"hour", "mobile", "userFeatures"}) @@ -61,7 +58,7 @@ public static void main(String[] args) { Dataset output = assembler.transform(dataset); System.out.println(output.select("features", "clicked").first()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java index b3b5953ee7bb..dd9d757dd683 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java @@ -17,9 +17,7 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Map; @@ -32,12 +30,13 @@ public class JavaVectorIndexerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaVectorIndexerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaVectorIndexerExample") + .getOrCreate(); // $example on$ - Dataset data = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); VectorIndexer indexer = new VectorIndexer() .setInputCol("features") @@ -57,6 +56,6 @@ public static void main(String[] args) { Dataset indexedData = indexerModel.transform(data); indexedData.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java index 2ae57c3577ef..24959c0e10f2 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java @@ -17,14 +17,13 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ +import java.util.List; + import com.google.common.collect.Lists; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.attribute.Attribute; import org.apache.spark.ml.attribute.AttributeGroup; import org.apache.spark.ml.attribute.NumericAttribute; @@ -38,9 +37,10 @@ public class JavaVectorSlicerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaVectorSlicerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaVectorSlicerExample") + .getOrCreate(); // $example on$ Attribute[] attrs = new Attribute[]{ @@ -50,13 +50,13 @@ public static void main(String[] args) { }; AttributeGroup group = new AttributeGroup("userFeatures", attrs); - JavaRDD jrdd = jsc.parallelize(Lists.newArrayList( + List data = Lists.newArrayList( RowFactory.create(Vectors.sparse(3, new int[]{0, 1}, new double[]{-2.0, 2.3})), RowFactory.create(Vectors.dense(-2.0, 2.3, 0.0)) - )); + ); Dataset dataset = - jsql.createDataFrame(jrdd, (new StructType()).add(group.toStructField())); + spark.createDataFrame(data, (new StructType()).add(group.toStructField())); VectorSlicer vectorSlicer = new VectorSlicer() .setInputCol("userFeatures").setOutputCol("features"); @@ -68,7 +68,7 @@ public static void main(String[] args) { System.out.println(output.select("userFeatures", "features").first()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java index c5bb1eaaa344..9be6e6353adc 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java @@ -19,37 +19,35 @@ // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.Word2Vec; import org.apache.spark.ml.feature.Word2VecModel; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; // $example off$ public class JavaWord2VecExample { public static void main(String[] args) { - - SparkConf conf = new SparkConf().setAppName("JavaWord2VecExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaWord2VecExample") + .getOrCreate(); // $example on$ // Input data: Each row is a bag of words from a sentence or document. - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Arrays.asList("Hi I heard about Spark".split(" "))), RowFactory.create(Arrays.asList("I wish Java could use case classes".split(" "))), RowFactory.create(Arrays.asList("Logistic regression models are neat".split(" "))) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("text", new ArrayType(DataTypes.StringType, true), false, Metadata.empty()) }); - Dataset documentDF = sqlContext.createDataFrame(jrdd, schema); + Dataset documentDF = spark.createDataFrame(data, schema); // Learn a mapping from words to Vectors. Word2Vec word2Vec = new Word2Vec() @@ -64,6 +62,6 @@ public static void main(String[] args) { } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index 354a5306ed45..55e591d0ce16 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -21,14 +21,12 @@ import java.util.Arrays; import java.util.List; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; public class JavaSparkSQL { public static class Person implements Serializable { @@ -53,13 +51,15 @@ public void setAge(int age) { } public static void main(String[] args) throws Exception { - SparkConf sparkConf = new SparkConf().setAppName("JavaSparkSQL"); - JavaSparkContext ctx = new JavaSparkContext(sparkConf); - SQLContext sqlContext = new SQLContext(ctx); + SparkSession spark = SparkSession + .builder() + .appName("JavaSparkSQL") + .getOrCreate(); System.out.println("=== Data source: RDD ==="); // Load a text file and convert each line to a Java Bean. - JavaRDD people = ctx.textFile("examples/src/main/resources/people.txt").map( + String file = "examples/src/main/resources/people.txt"; + JavaRDD people = spark.read().text(file).javaRDD().map( new Function() { @Override public Person call(String line) { @@ -73,13 +73,12 @@ public Person call(String line) { } }); - // Apply a schema to an RDD of Java Beans and register it as a table. - Dataset schemaPeople = sqlContext.createDataFrame(people, Person.class); - schemaPeople.registerTempTable("people"); + // Apply a schema to an RDD of Java Beans and create a temporary view + Dataset schemaPeople = spark.createDataFrame(people, Person.class); + schemaPeople.createOrReplaceTempView("people"); - // SQL can be run over RDDs that have been registered as tables. - Dataset teenagers = - sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + // SQL can be run over RDDs which backs a temporary view. + Dataset teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); // The results of SQL queries are DataFrames and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. @@ -100,12 +99,12 @@ public String call(Row row) { // Read in the parquet file created above. // Parquet files are self-describing so the schema is preserved. // The result of loading a parquet file is also a DataFrame. - Dataset parquetFile = sqlContext.read().parquet("people.parquet"); + Dataset parquetFile = spark.read().parquet("people.parquet"); - //Parquet files can also be registered as tables and then used in SQL statements. - parquetFile.registerTempTable("parquetFile"); + // A temporary view can be created by using Parquet files and then used in SQL statements. + parquetFile.createOrReplaceTempView("parquetFile"); Dataset teenagers2 = - sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); + spark.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); teenagerNames = teenagers2.toJavaRDD().map(new Function() { @Override public String call(Row row) { @@ -121,7 +120,7 @@ public String call(Row row) { // The path can be either a single text file or a directory storing text files. String path = "examples/src/main/resources/people.json"; // Create a DataFrame from the file(s) pointed by path - Dataset peopleFromJsonFile = sqlContext.read().json(path); + Dataset peopleFromJsonFile = spark.read().json(path); // Because the schema of a JSON dataset is automatically inferred, to write queries, // it is better to take a look at what is the schema. @@ -131,12 +130,11 @@ public String call(Row row) { // |-- age: IntegerType // |-- name: StringType - // Register this DataFrame as a table. - peopleFromJsonFile.registerTempTable("people"); + // Creates a temporary view using the DataFrame + peopleFromJsonFile.createOrReplaceTempView("people"); - // SQL statements can be run by using the sql methods provided by sqlContext. - Dataset teenagers3 = - sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + // SQL statements can be run by using the sql methods provided by `spark` + Dataset teenagers3 = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); // The results of SQL queries are DataFrame and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. @@ -152,8 +150,9 @@ public String call(Row row) { // a RDD[String] storing one JSON object per string. List jsonData = Arrays.asList( "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); - JavaRDD anotherPeopleRDD = ctx.parallelize(jsonData); - Dataset peopleFromJsonRDD = sqlContext.read().json(anotherPeopleRDD.rdd()); + JavaRDD anotherPeopleRDD = spark + .createDataFrame(jsonData, String.class).toJSON().javaRDD(); + Dataset peopleFromJsonRDD = spark.read().json(anotherPeopleRDD); // Take a look at the schema of this new DataFrame. peopleFromJsonRDD.printSchema(); @@ -164,9 +163,9 @@ public String call(Row row) { // | |-- state: StringType // |-- name: StringType - peopleFromJsonRDD.registerTempTable("people2"); + peopleFromJsonRDD.createOrReplaceTempView("people2"); - Dataset peopleWithCity = sqlContext.sql("SELECT name, address.city FROM people2"); + Dataset peopleWithCity = spark.sql("SELECT name, address.city FROM people2"); List nameAndCity = peopleWithCity.toJavaRDD().map(new Function() { @Override public String call(Row row) { @@ -177,6 +176,6 @@ public String call(Row row) { System.out.println(name); } - ctx.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java index 7aa8862761d2..b8e9e125ba59 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java @@ -22,14 +22,13 @@ import java.util.regex.Pattern; import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.VoidFunction2; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.api.java.StorageLevels; import org.apache.spark.streaming.Durations; import org.apache.spark.streaming.Time; @@ -82,7 +81,7 @@ public Iterator call(String x) { words.foreachRDD(new VoidFunction2, Time>() { @Override public void call(JavaRDD rdd, Time time) { - SQLContext sqlContext = JavaSQLContextSingleton.getInstance(rdd.context()); + SparkSession spark = JavaSparkSessionSingleton.getInstance(rdd.context().getConf()); // Convert JavaRDD[String] to JavaRDD[bean class] to DataFrame JavaRDD rowRDD = rdd.map(new Function() { @@ -93,14 +92,14 @@ public JavaRecord call(String word) { return record; } }); - Dataset wordsDataFrame = sqlContext.createDataFrame(rowRDD, JavaRecord.class); + Dataset wordsDataFrame = spark.createDataFrame(rowRDD, JavaRecord.class); - // Register as table - wordsDataFrame.registerTempTable("words"); + // Creates a temporary view using the DataFrame + wordsDataFrame.createOrReplaceTempView("words"); // Do word count on table using SQL and print it Dataset wordCountsDataFrame = - sqlContext.sql("select word, count(*) as total from words group by word"); + spark.sql("select word, count(*) as total from words group by word"); System.out.println("========= " + time + "========="); wordCountsDataFrame.show(); } @@ -111,12 +110,15 @@ public JavaRecord call(String word) { } } -/** Lazily instantiated singleton instance of SQLContext */ -class JavaSQLContextSingleton { - private static transient SQLContext instance = null; - public static SQLContext getInstance(SparkContext sparkContext) { +/** Lazily instantiated singleton instance of SparkSession */ +class JavaSparkSessionSingleton { + private static transient SparkSession instance = null; + public static SparkSession getInstance(SparkConf sparkConf) { if (instance == null) { - instance = new SQLContext(sparkContext); + instance = SparkSession + .builder() + .config(sparkConf) + .getOrCreate(); } return instance; } diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 205ca02962be..f07020b50397 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -17,7 +17,7 @@ """ This is an example implementation of ALS for learning how to use Spark. Please refer to -ALS in pyspark.mllib.recommendation for more conventional use. +pyspark.ml.recommendation.ALS for more conventional use. This example requires numpy (http://www.numpy.org/) """ @@ -59,7 +59,7 @@ def update(i, vec, mat, ratings): """ print("""WARN: This is a naive implementation of ALS and is given as an - example. Please use the ALS method found in pyspark.mllib.recommendation for more + example. Please use pyspark.ml.recommendation.ALS for more conventional use.""", file=sys.stderr) sc = SparkContext(appName="PythonALS") diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index 0ea7cfb7025a..3426e491dc74 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -17,8 +17,8 @@ """ The K-means algorithm written from scratch against PySpark. In practice, -one may prefer to use the KMeans algorithm in MLlib, as shown in -examples/src/main/python/mllib/kmeans.py. +one may prefer to use the KMeans algorithm in ML, as shown in +examples/src/main/python/ml/kmeans_example.py. This example requires NumPy (http://www.numpy.org/). """ @@ -52,8 +52,8 @@ def closestPoint(p, centers): exit(-1) print("""WARN: This is a naive implementation of KMeans Clustering and is given - as an example! Please refer to examples/src/main/python/mllib/kmeans.py for an example on - how to use MLlib's KMeans implementation.""", file=sys.stderr) + as an example! Please refer to examples/src/main/python/ml/kmeans_example.py for an + example on how to use ML's KMeans implementation.""", file=sys.stderr) sc = SparkContext(appName="PythonKMeans") lines = sc.textFile(sys.argv[1]) diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index b318b7d87bfd..7d33be7e81d7 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -20,7 +20,7 @@ to act on batches of input data using efficient matrix operations. In practice, one may prefer to use the LogisticRegression algorithm in -MLlib, as shown in examples/src/main/python/mllib/logistic_regression.py. +ML, as shown in examples/src/main/python/ml/logistic_regression_with_elastic_net.py. """ from __future__ import print_function @@ -51,8 +51,9 @@ def readPointBatch(iterator): exit(-1) print("""WARN: This is a naive implementation of Logistic Regression and is - given as an example! Please refer to examples/src/main/python/mllib/logistic_regression.py - to see how MLlib's implementation is used.""", file=sys.stderr) + given as an example! + Please refer to examples/src/main/python/ml/logistic_regression_with_elastic_net.py + to see how ML's implementation is used.""", file=sys.stderr) sc = SparkContext(appName="PythonLR") points = sc.textFile(sys.argv[1]).mapPartitions(readPointBatch).cache() diff --git a/examples/src/main/python/ml/aft_survival_regression.py b/examples/src/main/python/ml/aft_survival_regression.py index 0ee01fd8258d..9879679829d4 100644 --- a/examples/src/main/python/ml/aft_survival_regression.py +++ b/examples/src/main/python/ml/aft_survival_regression.py @@ -17,19 +17,26 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.regression import AFTSurvivalRegression from pyspark.mllib.linalg import Vectors # $example off$ +from pyspark.sql import SparkSession + +""" +An example demonstrating aft survival regression. +Run with: + bin/spark-submit examples/src/main/python/ml/aft_survival_regression.py +""" if __name__ == "__main__": - sc = SparkContext(appName="AFTSurvivalRegressionExample") - sqlContext = SQLContext(sc) + spark = SparkSession \ + .builder \ + .appName("PythonAFTSurvivalRegressionExample") \ + .getOrCreate() # $example on$ - training = sqlContext.createDataFrame([ + training = spark.createDataFrame([ (1.218, 1.0, Vectors.dense(1.560, -0.605)), (2.949, 0.0, Vectors.dense(0.346, 2.158)), (3.627, 0.0, Vectors.dense(1.380, 0.231)), @@ -48,4 +55,4 @@ model.transform(training).show(truncate=False) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/als_example.py b/examples/src/main/python/ml/als_example.py index 922173308c6a..1a979ff5b5be 100644 --- a/examples/src/main/python/ml/als_example.py +++ b/examples/src/main/python/ml/als_example.py @@ -17,8 +17,11 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext +import sys +if sys.version >= '3': + long = int + +from pyspark.sql import SparkSession # $example on$ from pyspark.ml.evaluation import RegressionEvaluator @@ -27,15 +30,17 @@ # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="ALSExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("ALSExample")\ + .getOrCreate() # $example on$ - lines = sc.textFile("data/mllib/als/sample_movielens_ratings.txt") - parts = lines.map(lambda l: l.split("::")) + lines = spark.read.text("data/mllib/als/sample_movielens_ratings.txt").rdd + parts = lines.map(lambda row: row.value.split("::")) ratingsRDD = parts.map(lambda p: Row(userId=int(p[0]), movieId=int(p[1]), rating=float(p[2]), timestamp=long(p[3]))) - ratings = sqlContext.createDataFrame(ratingsRDD) + ratings = spark.createDataFrame(ratingsRDD) (training, test) = ratings.randomSplit([0.8, 0.2]) # Build the recommendation model using ALS on the training data @@ -43,13 +48,10 @@ model = als.fit(training) # Evaluate the model by computing the RMSE on the test data - rawPredictions = model.transform(test) - predictions = rawPredictions\ - .withColumn("rating", rawPredictions.rating.cast("double"))\ - .withColumn("prediction", rawPredictions.prediction.cast("double")) - evaluator =\ - RegressionEvaluator(metricName="rmse", labelCol="rating", predictionCol="prediction") + predictions = model.transform(test) + evaluator = RegressionEvaluator(metricName="rmse", labelCol="rating", + predictionCol="prediction") rmse = evaluator.evaluate(predictions) print("Root-mean-square error = " + str(rmse)) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/binarizer_example.py b/examples/src/main/python/ml/binarizer_example.py index 317cfa638a5a..4224a27dbef0 100644 --- a/examples/src/main/python/ml/binarizer_example.py +++ b/examples/src/main/python/ml/binarizer_example.py @@ -17,18 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import Binarizer # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="BinarizerExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("BinarizerExample")\ + .getOrCreate() # $example on$ - continuousDataFrame = sqlContext.createDataFrame([ + continuousDataFrame = spark.createDataFrame([ (0, 0.1), (1, 0.8), (2, 0.2) @@ -40,4 +41,4 @@ print(binarized_feature) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/bisecting_k_means_example.py b/examples/src/main/python/ml/bisecting_k_means_example.py index e6f6bfd7e84e..ee0399ac5eb2 100644 --- a/examples/src/main/python/ml/bisecting_k_means_example.py +++ b/examples/src/main/python/ml/bisecting_k_means_example.py @@ -17,41 +17,40 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ -from pyspark.ml.clustering import BisectingKMeans, BisectingKMeansModel -from pyspark.mllib.linalg import VectorUDT, _convert_to_vector, Vectors -from pyspark.mllib.linalg import Vectors -from pyspark.sql.types import Row +from pyspark.ml.clustering import BisectingKMeans # $example off$ -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession """ -A simple example demonstrating a bisecting k-means clustering. +An example demonstrating bisecting k-means clustering. +Run with: + bin/spark-submit examples/src/main/python/ml/bisecting_k_means_example.py """ if __name__ == "__main__": - - sc = SparkContext(appName="PythonBisectingKMeansExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("PythonBisectingKMeansExample")\ + .getOrCreate() # $example on$ - data = sc.textFile("data/mllib/kmeans_data.txt") - parsed = data.map(lambda l: Row(features=Vectors.dense([float(x) for x in l.split(' ')]))) - training = sqlContext.createDataFrame(parsed) - - kmeans = BisectingKMeans().setK(2).setSeed(1).setFeaturesCol("features") + # Loads data. + dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") - model = kmeans.fit(training) + # Trains a bisecting k-means model. + bkm = BisectingKMeans().setK(2).setSeed(1) + model = bkm.fit(dataset) - # Evaluate clustering - cost = model.computeCost(training) - print("Bisecting K-means Cost = " + str(cost)) + # Evaluate clustering. + cost = model.computeCost(dataset) + print("Within Set Sum of Squared Errors = " + str(cost)) - centers = model.clusterCenters() + # Shows the result. print("Cluster Centers: ") + centers = model.clusterCenters() for center in centers: print(center) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/bucketizer_example.py b/examples/src/main/python/ml/bucketizer_example.py index 4304255f350d..8177e560ddef 100644 --- a/examples/src/main/python/ml/bucketizer_example.py +++ b/examples/src/main/python/ml/bucketizer_example.py @@ -17,21 +17,22 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import Bucketizer # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="BucketizerExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("BucketizerExample")\ + .getOrCreate() # $example on$ splits = [-float("inf"), -0.5, 0.0, 0.5, float("inf")] data = [(-0.5,), (-0.3,), (0.0,), (0.2,)] - dataFrame = sqlContext.createDataFrame(data, ["features"]) + dataFrame = spark.createDataFrame(data, ["features"]) bucketizer = Bucketizer(splits=splits, inputCol="features", outputCol="bucketedFeatures") @@ -40,4 +41,4 @@ bucketedData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/chisq_selector_example.py b/examples/src/main/python/ml/chisq_selector_example.py index 997a50473536..8bafb942e0d2 100644 --- a/examples/src/main/python/ml/chisq_selector_example.py +++ b/examples/src/main/python/ml/chisq_selector_example.py @@ -17,19 +17,20 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import ChiSqSelector from pyspark.mllib.linalg import Vectors # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="ChiSqSelectorExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("ChiSqSelectorExample")\ + .getOrCreate() # $example on$ - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ (7, Vectors.dense([0.0, 0.0, 18.0, 1.0]), 1.0,), (8, Vectors.dense([0.0, 1.0, 12.0, 0.0]), 0.0,), (9, Vectors.dense([1.0, 0.0, 15.0, 0.1]), 0.0,)], ["id", "features", "clicked"]) @@ -41,4 +42,4 @@ result.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/count_vectorizer_example.py b/examples/src/main/python/ml/count_vectorizer_example.py index e839f645f70b..38cfac82fbe2 100644 --- a/examples/src/main/python/ml/count_vectorizer_example.py +++ b/examples/src/main/python/ml/count_vectorizer_example.py @@ -17,19 +17,20 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import CountVectorizer # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="CountVectorizerExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("CountVectorizerExample")\ + .getOrCreate() # $example on$ # Input data: Each row is a bag of words with a ID. - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ (0, "a b c".split(" ")), (1, "a b b c a".split(" ")) ], ["id", "words"]) @@ -41,4 +42,4 @@ result.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/cross_validator.py b/examples/src/main/python/ml/cross_validator.py index 5f0ef20218c4..a41df6cf946f 100644 --- a/examples/src/main/python/ml/cross_validator.py +++ b/examples/src/main/python/ml/cross_validator.py @@ -17,15 +17,14 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression from pyspark.ml.evaluation import BinaryClassificationEvaluator from pyspark.ml.feature import HashingTF, Tokenizer from pyspark.ml.tuning import CrossValidator, ParamGridBuilder -from pyspark.sql import Row, SQLContext # $example off$ +from pyspark.sql import Row, SparkSession """ A simple example demonstrating model selection using CrossValidator. @@ -36,25 +35,26 @@ """ if __name__ == "__main__": - sc = SparkContext(appName="CrossValidatorExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("CrossValidatorExample")\ + .getOrCreate() # $example on$ # Prepare training documents, which are labeled. - LabeledDocument = Row("id", "text", "label") - training = sc.parallelize([(0, "a b c d e spark", 1.0), - (1, "b d", 0.0), - (2, "spark f g h", 1.0), - (3, "hadoop mapreduce", 0.0), - (4, "b spark who", 1.0), - (5, "g d a y", 0.0), - (6, "spark fly", 1.0), - (7, "was mapreduce", 0.0), - (8, "e spark program", 1.0), - (9, "a e c l", 0.0), - (10, "spark compile", 1.0), - (11, "hadoop software", 0.0) - ]) \ - .map(lambda x: LabeledDocument(*x)).toDF() + training = spark.createDataFrame([ + (0, "a b c d e spark", 1.0), + (1, "b d", 0.0), + (2, "spark f g h", 1.0), + (3, "hadoop mapreduce", 0.0), + (4, "b spark who", 1.0), + (5, "g d a y", 0.0), + (6, "spark fly", 1.0), + (7, "was mapreduce", 0.0), + (8, "e spark program", 1.0), + (9, "a e c l", 0.0), + (10, "spark compile", 1.0), + (11, "hadoop software", 0.0) + ], ["id", "text", "label"]) # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. tokenizer = Tokenizer(inputCol="text", outputCol="words") @@ -82,12 +82,12 @@ cvModel = crossval.fit(training) # Prepare test documents, which are unlabeled. - Document = Row("id", "text") - test = sc.parallelize([(4L, "spark i j k"), - (5L, "l m n"), - (6L, "mapreduce spark"), - (7L, "apache hadoop")]) \ - .map(lambda x: Document(*x)).toDF() + test = spark.createDataFrame([ + (4L, "spark i j k"), + (5L, "l m n"), + (6L, "mapreduce spark"), + (7L, "apache hadoop") + ], ["id", "text"]) # Make predictions on test documents. cvModel uses the best model found (lrModel). prediction = cvModel.transform(test) @@ -96,4 +96,4 @@ print(row) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/dataframe_example.py b/examples/src/main/python/ml/dataframe_example.py index d2644ca33565..a7d8b9056dff 100644 --- a/examples/src/main/python/ml/dataframe_example.py +++ b/examples/src/main/python/ml/dataframe_example.py @@ -26,16 +26,17 @@ import tempfile import shutil -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession from pyspark.mllib.stat import Statistics if __name__ == "__main__": if len(sys.argv) > 2: print("Usage: dataframe_example.py ", file=sys.stderr) exit(-1) - sc = SparkContext(appName="DataFrameExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("DataFrameExample")\ + .getOrCreate() if len(sys.argv) == 2: input = sys.argv[1] else: @@ -43,7 +44,7 @@ # Load input data print("Loading LIBSVM file with UDT from " + input + ".") - df = sqlContext.read.format("libsvm").load(input).cache() + df = spark.read.format("libsvm").load(input).cache() print("Schema from LIBSVM:") df.printSchema() print("Loaded training data as a DataFrame with " + @@ -54,7 +55,7 @@ labelSummary.show() # Convert features column to an RDD of vectors. - features = df.select("features").map(lambda r: r.features) + features = df.select("features").rdd.map(lambda r: r.features) summary = Statistics.colStats(features) print("Selected features column with average values:\n" + str(summary.mean())) @@ -67,9 +68,9 @@ # Load the records back. print("Loading Parquet file with UDT from " + tempdir) - newDF = sqlContext.read.parquet(tempdir) + newDF = spark.read.parquet(tempdir) print("Schema from Parquet:") newDF.printSchema() shutil.rmtree(tempdir) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/dct_example.py b/examples/src/main/python/ml/dct_example.py index 264d47f404cb..e36fcdeaeed2 100644 --- a/examples/src/main/python/ml/dct_example.py +++ b/examples/src/main/python/ml/dct_example.py @@ -17,19 +17,20 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import DCT from pyspark.mllib.linalg import Vectors # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="DCTExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("DCTExample")\ + .getOrCreate() # $example on$ - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ (Vectors.dense([0.0, 1.0, -2.0, 3.0]),), (Vectors.dense([-1.0, 2.0, 4.0, -7.0]),), (Vectors.dense([14.0, -2.0, -5.0, 1.0]),)], ["features"]) @@ -42,4 +43,4 @@ print(dcts) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/decision_tree_classification_example.py b/examples/src/main/python/ml/decision_tree_classification_example.py index 86bdc65392bb..9b40b701ecaa 100644 --- a/examples/src/main/python/ml/decision_tree_classification_example.py +++ b/examples/src/main/python/ml/decision_tree_classification_example.py @@ -21,20 +21,22 @@ from __future__ import print_function # $example on$ -from pyspark import SparkContext, SQLContext from pyspark.ml import Pipeline from pyspark.ml.classification import DecisionTreeClassifier from pyspark.ml.feature import StringIndexer, VectorIndexer from pyspark.ml.evaluation import MulticlassClassificationEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="decision_tree_classification_example") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("decision_tree_classification_example")\ + .getOrCreate() # $example on$ # Load the data stored in LIBSVM format as a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Index labels, adding metadata to the label column. # Fit on whole dataset to include all labels in index. @@ -72,3 +74,5 @@ # summary only print(treeModel) # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/decision_tree_regression_example.py b/examples/src/main/python/ml/decision_tree_regression_example.py index 8e20d5d8572a..b734d4974a4f 100644 --- a/examples/src/main/python/ml/decision_tree_regression_example.py +++ b/examples/src/main/python/ml/decision_tree_regression_example.py @@ -20,21 +20,23 @@ """ from __future__ import print_function -from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.regression import DecisionTreeRegressor from pyspark.ml.feature import VectorIndexer from pyspark.ml.evaluation import RegressionEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="decision_tree_classification_example") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("decision_tree_classification_example")\ + .getOrCreate() # $example on$ # Load the data stored in LIBSVM format as a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Automatically identify categorical features, and index them. # We specify maxCategories so features with > 4 distinct values are treated as continuous. @@ -69,3 +71,5 @@ # summary only print(treeModel) # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/elementwise_product_example.py b/examples/src/main/python/ml/elementwise_product_example.py index c85cb0d89543..41727edcdb09 100644 --- a/examples/src/main/python/ml/elementwise_product_example.py +++ b/examples/src/main/python/ml/elementwise_product_example.py @@ -17,23 +17,24 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import ElementwiseProduct from pyspark.mllib.linalg import Vectors # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="ElementwiseProductExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("ElementwiseProductExample")\ + .getOrCreate() # $example on$ data = [(Vectors.dense([1.0, 2.0, 3.0]),), (Vectors.dense([4.0, 5.0, 6.0]),)] - df = sqlContext.createDataFrame(data, ["vector"]) + df = spark.createDataFrame(data, ["vector"]) transformer = ElementwiseProduct(scalingVec=Vectors.dense([0.0, 1.0, 2.0]), inputCol="vector", outputCol="transformedVector") transformer.transform(df).show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/estimator_transformer_param_example.py b/examples/src/main/python/ml/estimator_transformer_param_example.py index 9a8993dac4f6..0fcae0e3fc22 100644 --- a/examples/src/main/python/ml/estimator_transformer_param_example.py +++ b/examples/src/main/python/ml/estimator_transformer_param_example.py @@ -18,20 +18,22 @@ """ Estimator Transformer Param Example. """ -from pyspark import SparkContext, SQLContext + # $example on$ from pyspark.mllib.linalg import Vectors from pyspark.ml.classification import LogisticRegression # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - - sc = SparkContext(appName="EstimatorTransformerParamExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("EstimatorTransformerParamExample")\ + .getOrCreate() # $example on$ # Prepare training data from a list of (label, features) tuples. - training = sqlContext.createDataFrame([ + training = spark.createDataFrame([ (1.0, Vectors.dense([0.0, 1.1, 0.1])), (0.0, Vectors.dense([2.0, 1.0, -1.0])), (0.0, Vectors.dense([2.0, 1.3, 1.0])), @@ -69,7 +71,7 @@ print model2.extractParamMap() # Prepare test data - test = sqlContext.createDataFrame([ + test = spark.createDataFrame([ (1.0, Vectors.dense([-1.0, 1.5, 1.3])), (0.0, Vectors.dense([3.0, 2.0, -0.1])), (1.0, Vectors.dense([0.0, 2.2, -1.5]))], ["label", "features"]) @@ -84,4 +86,4 @@ print row # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/gaussian_mixture_example.py b/examples/src/main/python/ml/gaussian_mixture_example.py new file mode 100644 index 000000000000..2ca13d68f689 --- /dev/null +++ b/examples/src/main/python/ml/gaussian_mixture_example.py @@ -0,0 +1,48 @@ +# +# 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. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.clustering import GaussianMixture +# $example off$ +from pyspark.sql import SparkSession + +""" +A simple example demonstrating Gaussian Mixture Model (GMM). +Run with: + bin/spark-submit examples/src/main/python/ml/gaussian_mixture_example.py +""" + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("PythonGuassianMixtureExample")\ + .getOrCreate() + + # $example on$ + # loads data + dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") + + gmm = GaussianMixture().setK(2) + model = gmm.fit(dataset) + + print("Gaussians: ") + model.gaussiansDF.show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/generalized_linear_regression_example.py b/examples/src/main/python/ml/generalized_linear_regression_example.py new file mode 100644 index 000000000000..796752a60f3a --- /dev/null +++ b/examples/src/main/python/ml/generalized_linear_regression_example.py @@ -0,0 +1,66 @@ +# +# 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. +# + +from __future__ import print_function + +from pyspark.sql import SparkSession +# $example on$ +from pyspark.ml.regression import GeneralizedLinearRegression +# $example off$ + +""" +An example demonstrating generalized linear regression. +Run with: + bin/spark-submit examples/src/main/python/ml/generalized_linear_regression_example.py +""" + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("GeneralizedLinearRegressionExample")\ + .getOrCreate() + + # $example on$ + # Load training data + dataset = spark.read.format("libsvm")\ + .load("data/mllib/sample_linear_regression_data.txt") + + glr = GeneralizedLinearRegression(family="gaussian", link="identity", maxIter=10, regParam=0.3) + + # Fit the model + model = glr.fit(dataset) + + # Print the coefficients and intercept for generalized linear regression model + print("Coefficients: " + str(model.coefficients)) + print("Intercept: " + str(model.intercept)) + + # Summarize the model over the training set and print out some metrics + summary = model.summary + print("Coefficient Standard Errors: " + str(summary.coefficientStandardErrors)) + print("T Values: " + str(summary.tValues)) + print("P Values: " + str(summary.pValues)) + print("Dispersion: " + str(summary.dispersion)) + print("Null Deviance: " + str(summary.nullDeviance)) + print("Residual Degree Of Freedom Null: " + str(summary.residualDegreeOfFreedomNull)) + print("Deviance: " + str(summary.deviance)) + print("Residual Degree Of Freedom: " + str(summary.residualDegreeOfFreedom)) + print("AIC: " + str(summary.aic)) + print("Deviance Residuals: ") + summary.residuals().show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py index f7e842f4b303..50026d7b7e3e 100644 --- a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py +++ b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py @@ -20,21 +20,23 @@ """ from __future__ import print_function -from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import GBTClassifier from pyspark.ml.feature import StringIndexer, VectorIndexer from pyspark.ml.evaluation import MulticlassClassificationEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="gradient_boosted_tree_classifier_example") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("gradient_boosted_tree_classifier_example")\ + .getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Index labels, adding metadata to the label column. # Fit on whole dataset to include all labels in index. @@ -72,4 +74,4 @@ print(gbtModel) # summary only # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py index f8b4de651c76..5dd2272748d7 100644 --- a/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py +++ b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py @@ -20,21 +20,23 @@ """ from __future__ import print_function -from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.regression import GBTRegressor from pyspark.ml.feature import VectorIndexer from pyspark.ml.evaluation import RegressionEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="gradient_boosted_tree_regressor_example") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("gradient_boosted_tree_regressor_example")\ + .getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Automatically identify categorical features, and index them. # Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -69,4 +71,4 @@ print(gbtModel) # summary only # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/index_to_string_example.py b/examples/src/main/python/ml/index_to_string_example.py index fb0ba2950bbd..523caac00c18 100644 --- a/examples/src/main/python/ml/index_to_string_example.py +++ b/examples/src/main/python/ml/index_to_string_example.py @@ -17,18 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ from pyspark.ml.feature import IndexToString, StringIndexer # $example off$ -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="IndexToStringExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("IndexToStringExample")\ + .getOrCreate() # $example on$ - df = sqlContext.createDataFrame( + df = spark.createDataFrame( [(0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")], ["id", "category"]) @@ -42,4 +43,4 @@ converted.select("id", "originalCategory").show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/kmeans_example.py b/examples/src/main/python/ml/kmeans_example.py index fa57a4d3ada1..4b8b7291f918 100644 --- a/examples/src/main/python/ml/kmeans_example.py +++ b/examples/src/main/python/ml/kmeans_example.py @@ -17,54 +17,45 @@ from __future__ import print_function -import sys +# $example on$ +from pyspark.ml.clustering import KMeans +# $example off$ -import numpy as np -from pyspark import SparkContext -from pyspark.ml.clustering import KMeans, KMeansModel -from pyspark.mllib.linalg import VectorUDT, _convert_to_vector -from pyspark.sql import SQLContext -from pyspark.sql.types import Row, StructField, StructType +from pyspark.sql import SparkSession """ -A simple example demonstrating a k-means clustering. +An example demonstrating k-means clustering. Run with: - bin/spark-submit examples/src/main/python/ml/kmeans_example.py + bin/spark-submit examples/src/main/python/ml/kmeans_example.py This example requires NumPy (http://www.numpy.org/). """ -def parseVector(line): - array = np.array([float(x) for x in line.split(' ')]) - return _convert_to_vector(array) - - if __name__ == "__main__": - FEATURES_COL = "features" + spark = SparkSession\ + .builder\ + .appName("PythonKMeansExample")\ + .getOrCreate() - if len(sys.argv) != 3: - print("Usage: kmeans_example.py ", file=sys.stderr) - exit(-1) - path = sys.argv[1] - k = sys.argv[2] + # $example on$ + # Loads data. + dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") - sc = SparkContext(appName="PythonKMeansExample") - sqlContext = SQLContext(sc) + # Trains a k-means model. + kmeans = KMeans().setK(2).setSeed(1) + model = kmeans.fit(dataset) - lines = sc.textFile(path) - data = lines.map(parseVector) - row_rdd = data.map(lambda x: Row(x)) - schema = StructType([StructField(FEATURES_COL, VectorUDT(), False)]) - df = sqlContext.createDataFrame(row_rdd, schema) + # Evaluate clustering by computing Within Set Sum of Squared Errors. + wssse = model.computeCost(dataset) + print("Within Set Sum of Squared Errors = " + str(wssse)) - kmeans = KMeans().setK(2).setSeed(1).setFeaturesCol(FEATURES_COL) - model = kmeans.fit(df) + # Shows the result. centers = model.clusterCenters() - print("Cluster Centers: ") for center in centers: print(center) + # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/lda_example.py b/examples/src/main/python/ml/lda_example.py new file mode 100644 index 000000000000..6ca56adf3cb1 --- /dev/null +++ b/examples/src/main/python/ml/lda_example.py @@ -0,0 +1,64 @@ +# +# 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. +# + + +from __future__ import print_function + +# $example on$ +from pyspark.ml.clustering import LDA +# $example off$ +from pyspark.sql import SparkSession + + +""" +An example demonstrating LDA. +Run with: + bin/spark-submit examples/src/main/python/ml/lda_example.py +""" + + +if __name__ == "__main__": + # Creates a SparkSession + spark = SparkSession \ + .builder \ + .appName("PythonKMeansExample") \ + .getOrCreate() + + # $example on$ + # Loads data. + dataset = spark.read.format("libsvm").load("data/mllib/sample_lda_libsvm_data.txt") + + # Trains a LDA model. + lda = LDA(k=10, maxIter=10) + model = lda.fit(dataset) + + ll = model.logLikelihood(dataset) + lp = model.logPerplexity(dataset) + print("The lower bound on the log likelihood of the entire corpus: " + str(ll)) + print("The upper bound bound on perplexity: " + str(lp)) + + # Describe topics. + topics = model.describeTopics(3) + print("The topics described by their top-weighted terms:") + topics.show(truncate=False) + + # Shows the result + transformed = model.transform(dataset) + transformed.show(truncate=False) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/linear_regression_with_elastic_net.py b/examples/src/main/python/ml/linear_regression_with_elastic_net.py index a4cd40cf2672..620ab5b87e59 100644 --- a/examples/src/main/python/ml/linear_regression_with_elastic_net.py +++ b/examples/src/main/python/ml/linear_regression_with_elastic_net.py @@ -17,19 +17,20 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.regression import LinearRegression # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="LinearRegressionWithElasticNet") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("LinearRegressionWithElasticNet")\ + .getOrCreate() # $example on$ # Load training data - training = sqlContext.read.format("libsvm")\ + training = spark.read.format("libsvm")\ .load("data/mllib/sample_linear_regression_data.txt") lr = LinearRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8) @@ -42,4 +43,4 @@ print("Intercept: " + str(lrModel.intercept)) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/logistic_regression_with_elastic_net.py b/examples/src/main/python/ml/logistic_regression_with_elastic_net.py index b0b1d27e13bb..33d0689f75cd 100644 --- a/examples/src/main/python/ml/logistic_regression_with_elastic_net.py +++ b/examples/src/main/python/ml/logistic_regression_with_elastic_net.py @@ -17,19 +17,20 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.classification import LogisticRegression # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="LogisticRegressionWithElasticNet") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("LogisticRegressionWithElasticNet")\ + .getOrCreate() # $example on$ # Load training data - training = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + training = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") lr = LogisticRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8) @@ -41,4 +42,4 @@ print("Intercept: " + str(lrModel.intercept)) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/max_abs_scaler_example.py b/examples/src/main/python/ml/max_abs_scaler_example.py index d9b69eef1cd8..ab91198b083d 100644 --- a/examples/src/main/python/ml/max_abs_scaler_example.py +++ b/examples/src/main/python/ml/max_abs_scaler_example.py @@ -17,18 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import MaxAbsScaler # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="MaxAbsScalerExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("MaxAbsScalerExample")\ + .getOrCreate() # $example on$ - dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") scaler = MaxAbsScaler(inputCol="features", outputCol="scaledFeatures") @@ -40,4 +41,4 @@ scaledData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/min_max_scaler_example.py b/examples/src/main/python/ml/min_max_scaler_example.py index 2f8e4ade468b..e3e7bc205b1e 100644 --- a/examples/src/main/python/ml/min_max_scaler_example.py +++ b/examples/src/main/python/ml/min_max_scaler_example.py @@ -17,18 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import MinMaxScaler # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="MinMaxScalerExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("MinMaxScalerExample")\ + .getOrCreate() # $example on$ - dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") scaler = MinMaxScaler(inputCol="features", outputCol="scaledFeatures") @@ -40,4 +41,4 @@ scaledData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/multilayer_perceptron_classification.py b/examples/src/main/python/ml/multilayer_perceptron_classification.py index f84588f547ff..8bededc14db4 100644 --- a/examples/src/main/python/ml/multilayer_perceptron_classification.py +++ b/examples/src/main/python/ml/multilayer_perceptron_classification.py @@ -17,21 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.classification import MultilayerPerceptronClassifier from pyspark.ml.evaluation import MulticlassClassificationEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - - sc = SparkContext(appName="multilayer_perceptron_classification_example") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder.appName("multilayer_perceptron_classification_example").getOrCreate() # $example on$ # Load training data - data = sqlContext.read.format("libsvm")\ + data = spark.read.format("libsvm")\ .load("data/mllib/sample_multiclass_classification_data.txt") # Split the data into train and test splits = data.randomSplit([0.6, 0.4], 1234) @@ -52,4 +50,4 @@ print("Precision:" + str(evaluator.evaluate(predictionAndLabels))) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/n_gram_example.py b/examples/src/main/python/ml/n_gram_example.py index f2d85f53e721..9ac07f2c8ee2 100644 --- a/examples/src/main/python/ml/n_gram_example.py +++ b/examples/src/main/python/ml/n_gram_example.py @@ -17,18 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import NGram # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="NGramExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("NGramExample")\ + .getOrCreate() # $example on$ - wordDataFrame = sqlContext.createDataFrame([ + wordDataFrame = spark.createDataFrame([ (0, ["Hi", "I", "heard", "about", "Spark"]), (1, ["I", "wish", "Java", "could", "use", "case", "classes"]), (2, ["Logistic", "regression", "models", "are", "neat"]) @@ -39,4 +40,4 @@ print(ngrams_label) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/naive_bayes_example.py b/examples/src/main/python/ml/naive_bayes_example.py index db8fbea9bf9b..89255a2bae64 100644 --- a/examples/src/main/python/ml/naive_bayes_example.py +++ b/examples/src/main/python/ml/naive_bayes_example.py @@ -17,21 +17,21 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.classification import NaiveBayes from pyspark.ml.evaluation import MulticlassClassificationEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - - sc = SparkContext(appName="naive_bayes_example") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("naive_bayes_example")\ + .getOrCreate() # $example on$ # Load training data - data = sqlContext.read.format("libsvm") \ + data = spark.read.format("libsvm") \ .load("data/mllib/sample_libsvm_data.txt") # Split the data into train and test splits = data.randomSplit([0.6, 0.4], 1234) @@ -50,4 +50,4 @@ print("Precision:" + str(evaluator.evaluate(predictionAndLabels))) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/normalizer_example.py b/examples/src/main/python/ml/normalizer_example.py index d490221474c2..19012f51f402 100644 --- a/examples/src/main/python/ml/normalizer_example.py +++ b/examples/src/main/python/ml/normalizer_example.py @@ -17,18 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import Normalizer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="NormalizerExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("NormalizerExample")\ + .getOrCreate() # $example on$ - dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Normalize each Vector using $L^1$ norm. normalizer = Normalizer(inputCol="features", outputCol="normFeatures", p=1.0) @@ -40,4 +41,4 @@ lInfNormData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/one_vs_rest_example.py b/examples/src/main/python/ml/one_vs_rest_example.py new file mode 100644 index 000000000000..971156d0dd29 --- /dev/null +++ b/examples/src/main/python/ml/one_vs_rest_example.py @@ -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. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.classification import LogisticRegression, OneVsRest +from pyspark.ml.evaluation import MulticlassClassificationEvaluator +# $example off$ +from pyspark.sql import SparkSession + +""" +An example of Multiclass to Binary Reduction with One Vs Rest, +using Logistic Regression as the base classifier. +Run with: + bin/spark-submit examples/src/main/python/ml/one_vs_rest_example.py +""" + + +if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("PythonOneVsRestExample") \ + .getOrCreate() + + # $example on$ + # load data file. + inputData = spark.read.format("libsvm") \ + .load("data/mllib/sample_multiclass_classification_data.txt") + + # generate the train/test split. + (train, test) = inputData.randomSplit([0.8, 0.2]) + + # instantiate the base classifier. + lr = LogisticRegression(maxIter=10, tol=1E-6, fitIntercept=True) + + # instantiate the One Vs Rest Classifier. + ovr = OneVsRest(classifier=lr) + + # train the multiclass model. + ovrModel = ovr.fit(train) + + # score the model on test data. + predictions = ovrModel.transform(test) + + # obtain evaluator. + evaluator = MulticlassClassificationEvaluator(metricName="precision") + + # compute the classification error on test data. + precision = evaluator.evaluate(predictions) + print("Test Error : " + str(1 - precision)) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/onehot_encoder_example.py b/examples/src/main/python/ml/onehot_encoder_example.py index 0f94c26638d3..b9fceef68e70 100644 --- a/examples/src/main/python/ml/onehot_encoder_example.py +++ b/examples/src/main/python/ml/onehot_encoder_example.py @@ -17,18 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import OneHotEncoder, StringIndexer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="OneHotEncoderExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("OneHotEncoderExample")\ + .getOrCreate() # $example on$ - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ (0, "a"), (1, "b"), (2, "c"), @@ -45,4 +46,4 @@ encoded.select("id", "categoryVec").show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/pca_example.py b/examples/src/main/python/ml/pca_example.py index a17181f1b8a5..f1b3cdec7bd7 100644 --- a/examples/src/main/python/ml/pca_example.py +++ b/examples/src/main/python/ml/pca_example.py @@ -17,26 +17,27 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import PCA from pyspark.mllib.linalg import Vectors # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="PCAExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("PCAExample")\ + .getOrCreate() # $example on$ data = [(Vectors.sparse(5, [(1, 1.0), (3, 7.0)]),), (Vectors.dense([2.0, 0.0, 3.0, 4.0, 5.0]),), (Vectors.dense([4.0, 0.0, 0.0, 6.0, 7.0]),)] - df = sqlContext.createDataFrame(data, ["features"]) + df = spark.createDataFrame(data, ["features"]) pca = PCA(k=3, inputCol="features", outputCol="pcaFeatures") model = pca.fit(df) result = model.transform(df).select("pcaFeatures") result.show(truncate=False) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/pipeline_example.py b/examples/src/main/python/ml/pipeline_example.py index 3288568f0c28..bd10cfd7a252 100644 --- a/examples/src/main/python/ml/pipeline_example.py +++ b/examples/src/main/python/ml/pipeline_example.py @@ -18,21 +18,23 @@ """ Pipeline Example. """ -from pyspark import SparkContext, SQLContext + # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression from pyspark.ml.feature import HashingTF, Tokenizer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - - sc = SparkContext(appName="PipelineExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("PipelineExample")\ + .getOrCreate() # $example on$ # Prepare training documents from a list of (id, text, label) tuples. - training = sqlContext.createDataFrame([ + training = spark.createDataFrame([ (0L, "a b c d e spark", 1.0), (1L, "b d", 0.0), (2L, "spark f g h", 1.0), @@ -48,7 +50,7 @@ model = pipeline.fit(training) # Prepare test documents, which are unlabeled (id, text) tuples. - test = sqlContext.createDataFrame([ + test = spark.createDataFrame([ (4L, "spark i j k"), (5L, "l m n"), (6L, "mapreduce spark"), @@ -61,4 +63,4 @@ print(row) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/polynomial_expansion_example.py b/examples/src/main/python/ml/polynomial_expansion_example.py index 89f5cbe8f2f4..08882bcb256f 100644 --- a/examples/src/main/python/ml/polynomial_expansion_example.py +++ b/examples/src/main/python/ml/polynomial_expansion_example.py @@ -17,19 +17,20 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import PolynomialExpansion from pyspark.mllib.linalg import Vectors # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="PolynomialExpansionExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("PolynomialExpansionExample")\ + .getOrCreate() # $example on$ - df = sqlContext\ + df = spark\ .createDataFrame([(Vectors.dense([-2.0, 2.3]),), (Vectors.dense([0.0, 0.0]),), (Vectors.dense([0.6, -1.1]),)], @@ -40,4 +41,4 @@ print(expanded) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/quantile_discretizer_example.py b/examples/src/main/python/ml/quantile_discretizer_example.py new file mode 100644 index 000000000000..6ae7bb18f8c6 --- /dev/null +++ b/examples/src/main/python/ml/quantile_discretizer_example.py @@ -0,0 +1,39 @@ +# +# 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. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import QuantileDiscretizer +# $example off$ +from pyspark.sql import SparkSession + + +if __name__ == "__main__": + spark = SparkSession.builder.appName("PythonQuantileDiscretizerExample").getOrCreate() + + # $example on$ + data = [(0, 18.0,), (1, 19.0,), (2, 8.0,), (3, 5.0,), (4, 2.2,)] + dataFrame = spark.createDataFrame(data, ["id", "hour"]) + + discretizer = QuantileDiscretizer(numBuckets=3, inputCol="hour", outputCol="result") + + result = discretizer.fit(dataFrame).transform(dataFrame) + result.show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/random_forest_classifier_example.py b/examples/src/main/python/ml/random_forest_classifier_example.py index c3570438c51d..c618eaf60c2e 100644 --- a/examples/src/main/python/ml/random_forest_classifier_example.py +++ b/examples/src/main/python/ml/random_forest_classifier_example.py @@ -20,21 +20,23 @@ """ from __future__ import print_function -from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import RandomForestClassifier from pyspark.ml.feature import StringIndexer, VectorIndexer from pyspark.ml.evaluation import MulticlassClassificationEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="random_forest_classifier_example") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("random_forest_classifier_example")\ + .getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Index labels, adding metadata to the label column. # Fit on whole dataset to include all labels in index. @@ -72,4 +74,4 @@ print(rfModel) # summary only # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/random_forest_regressor_example.py b/examples/src/main/python/ml/random_forest_regressor_example.py index b77014f37923..3a793737dba8 100644 --- a/examples/src/main/python/ml/random_forest_regressor_example.py +++ b/examples/src/main/python/ml/random_forest_regressor_example.py @@ -20,21 +20,23 @@ """ from __future__ import print_function -from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.regression import RandomForestRegressor from pyspark.ml.feature import VectorIndexer from pyspark.ml.evaluation import RegressionEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="random_forest_regressor_example") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("random_forest_regressor_example")\ + .getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Automatically identify categorical features, and index them. # Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -69,4 +71,4 @@ print(rfModel) # summary only # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/rformula_example.py b/examples/src/main/python/ml/rformula_example.py index b544a1470076..d5df3ce4f591 100644 --- a/examples/src/main/python/ml/rformula_example.py +++ b/examples/src/main/python/ml/rformula_example.py @@ -17,18 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import RFormula # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="RFormulaExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("RFormulaExample")\ + .getOrCreate() # $example on$ - dataset = sqlContext.createDataFrame( + dataset = spark.createDataFrame( [(7, "US", 18, 1.0), (8, "CA", 12, 0.0), (9, "NZ", 15, 0.0)], @@ -41,4 +42,4 @@ output.select("features", "label").show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/simple_params_example.py b/examples/src/main/python/ml/simple_params_example.py index 2d6d115d54d0..c57e59d01b54 100644 --- a/examples/src/main/python/ml/simple_params_example.py +++ b/examples/src/main/python/ml/simple_params_example.py @@ -20,11 +20,10 @@ import pprint import sys -from pyspark import SparkContext from pyspark.ml.classification import LogisticRegression from pyspark.mllib.linalg import DenseVector from pyspark.mllib.regression import LabeledPoint -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession """ A simple example demonstrating ways to specify parameters for Estimators and Transformers. @@ -33,21 +32,20 @@ """ if __name__ == "__main__": - if len(sys.argv) > 1: - print("Usage: simple_params_example", file=sys.stderr) - exit(1) - sc = SparkContext(appName="PythonSimpleParamsExample") - sqlContext = SQLContext(sc) + spark = SparkSession \ + .builder \ + .appName("SimpleTextClassificationPipeline") \ + .getOrCreate() # prepare training data. # We create an RDD of LabeledPoints and convert them into a DataFrame. # A LabeledPoint is an Object with two fields named label and features # and Spark SQL identifies these fields and creates the schema appropriately. - training = sc.parallelize([ + training = spark.createDataFrame([ LabeledPoint(1.0, DenseVector([0.0, 1.1, 0.1])), LabeledPoint(0.0, DenseVector([2.0, 1.0, -1.0])), LabeledPoint(0.0, DenseVector([2.0, 1.3, 1.0])), - LabeledPoint(1.0, DenseVector([0.0, 1.2, -0.5]))]).toDF() + LabeledPoint(1.0, DenseVector([0.0, 1.2, -0.5]))]) # Create a LogisticRegression instance with maxIter = 10. # This instance is an Estimator. @@ -70,7 +68,7 @@ # We may alternatively specify parameters using a parameter map. # paramMap overrides all lr parameters set earlier. - paramMap = {lr.maxIter: 20, lr.thresholds: [0.45, 0.55], lr.probabilityCol: "myProbability"} + paramMap = {lr.maxIter: 20, lr.thresholds: [0.5, 0.5], lr.probabilityCol: "myProbability"} # Now learn a new model using the new parameters. model2 = lr.fit(training, paramMap) @@ -78,10 +76,10 @@ pprint.pprint(model2.extractParamMap()) # prepare test data. - test = sc.parallelize([ + test = spark.createDataFrame([ LabeledPoint(1.0, DenseVector([-1.0, 1.5, 1.3])), LabeledPoint(0.0, DenseVector([3.0, 2.0, -0.1])), - LabeledPoint(0.0, DenseVector([0.0, 2.2, -1.5]))]).toDF() + LabeledPoint(0.0, DenseVector([0.0, 2.2, -1.5]))]) # Make predictions on test data using the Transformer.transform() method. # LogisticRegressionModel.transform will only use the 'features' column. @@ -95,4 +93,4 @@ print("features=%s,label=%s -> prob=%s, prediction=%s" % (row.features, row.label, row.myProbability, row.prediction)) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index b4f06bf88874..886f43c0b08e 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -17,11 +17,10 @@ from __future__ import print_function -from pyspark import SparkContext from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression from pyspark.ml.feature import HashingTF, Tokenizer -from pyspark.sql import Row, SQLContext +from pyspark.sql import Row, SparkSession """ @@ -34,16 +33,18 @@ if __name__ == "__main__": - sc = SparkContext(appName="SimpleTextClassificationPipeline") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("SimpleTextClassificationPipeline")\ + .getOrCreate() # Prepare training documents, which are labeled. - LabeledDocument = Row("id", "text", "label") - training = sc.parallelize([(0, "a b c d e spark", 1.0), - (1, "b d", 0.0), - (2, "spark f g h", 1.0), - (3, "hadoop mapreduce", 0.0)]) \ - .map(lambda x: LabeledDocument(*x)).toDF() + training = spark.createDataFrame([ + (0, "a b c d e spark", 1.0), + (1, "b d", 0.0), + (2, "spark f g h", 1.0), + (3, "hadoop mapreduce", 0.0) + ], ["id", "text", "label"]) # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. tokenizer = Tokenizer(inputCol="text", outputCol="words") @@ -55,12 +56,12 @@ model = pipeline.fit(training) # Prepare test documents, which are unlabeled. - Document = Row("id", "text") - test = sc.parallelize([(4, "spark i j k"), - (5, "l m n"), - (6, "spark hadoop spark"), - (7, "apache hadoop")]) \ - .map(lambda x: Document(*x)).toDF() + test = spark.createDataFrame([ + (4, "spark i j k"), + (5, "l m n"), + (6, "spark hadoop spark"), + (7, "apache hadoop") + ], ["id", "text"]) # Make predictions on test documents and print columns of interest. prediction = model.transform(test) @@ -68,4 +69,4 @@ for row in selected.collect(): print(row) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/sql_transformer.py b/examples/src/main/python/ml/sql_transformer.py index 9575d728d815..0bf8f35720c9 100644 --- a/examples/src/main/python/ml/sql_transformer.py +++ b/examples/src/main/python/ml/sql_transformer.py @@ -17,18 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ from pyspark.ml.feature import SQLTransformer # $example off$ -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="SQLTransformerExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("SQLTransformerExample")\ + .getOrCreate() # $example on$ - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ (0, 1.0, 3.0), (2, 2.0, 5.0) ], ["id", "v1", "v2"]) @@ -37,4 +38,4 @@ sqlTrans.transform(df).show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/standard_scaler_example.py b/examples/src/main/python/ml/standard_scaler_example.py index ae7aa85005bc..c0027480e69b 100644 --- a/examples/src/main/python/ml/standard_scaler_example.py +++ b/examples/src/main/python/ml/standard_scaler_example.py @@ -17,18 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import StandardScaler # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="StandardScalerExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("StandardScalerExample")\ + .getOrCreate() # $example on$ - dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") scaler = StandardScaler(inputCol="features", outputCol="scaledFeatures", withStd=True, withMean=False) @@ -40,4 +41,4 @@ scaledData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/stopwords_remover_example.py b/examples/src/main/python/ml/stopwords_remover_example.py index 01f94af8ca75..395fdeffc537 100644 --- a/examples/src/main/python/ml/stopwords_remover_example.py +++ b/examples/src/main/python/ml/stopwords_remover_example.py @@ -17,18 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import StopWordsRemover # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="StopWordsRemoverExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("StopWordsRemoverExample")\ + .getOrCreate() # $example on$ - sentenceData = sqlContext.createDataFrame([ + sentenceData = spark.createDataFrame([ (0, ["I", "saw", "the", "red", "baloon"]), (1, ["Mary", "had", "a", "little", "lamb"]) ], ["label", "raw"]) @@ -37,4 +38,4 @@ remover.transform(sentenceData).show(truncate=False) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/string_indexer_example.py b/examples/src/main/python/ml/string_indexer_example.py index 58a8cb5d56b7..a328e040f563 100644 --- a/examples/src/main/python/ml/string_indexer_example.py +++ b/examples/src/main/python/ml/string_indexer_example.py @@ -17,18 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import StringIndexer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="StringIndexerExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("StringIndexerExample")\ + .getOrCreate() # $example on$ - df = sqlContext.createDataFrame( + df = spark.createDataFrame( [(0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")], ["id", "category"]) indexer = StringIndexer(inputCol="category", outputCol="categoryIndex") @@ -36,4 +37,4 @@ indexed.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/tf_idf_example.py b/examples/src/main/python/ml/tf_idf_example.py index 141324d45853..fb4ad992fb80 100644 --- a/examples/src/main/python/ml/tf_idf_example.py +++ b/examples/src/main/python/ml/tf_idf_example.py @@ -17,18 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ from pyspark.ml.feature import HashingTF, IDF, Tokenizer # $example off$ -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="TfIdfExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("TfIdfExample")\ + .getOrCreate() # $example on$ - sentenceData = sqlContext.createDataFrame([ + sentenceData = spark.createDataFrame([ (0, "Hi I heard about Spark"), (0, "I wish Java could use case classes"), (1, "Logistic regression models are neat") @@ -46,4 +47,4 @@ print(features_label) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/tokenizer_example.py b/examples/src/main/python/ml/tokenizer_example.py index ce9b225be535..e61ec920d228 100644 --- a/examples/src/main/python/ml/tokenizer_example.py +++ b/examples/src/main/python/ml/tokenizer_example.py @@ -17,18 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import Tokenizer, RegexTokenizer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="TokenizerExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("TokenizerExample")\ + .getOrCreate() # $example on$ - sentenceDataFrame = sqlContext.createDataFrame([ + sentenceDataFrame = spark.createDataFrame([ (0, "Hi I heard about Spark"), (1, "I wish Java could use case classes"), (2, "Logistic,regression,models,are,neat") @@ -41,4 +42,4 @@ # alternatively, pattern="\\w+", gaps(False) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/train_validation_split.py b/examples/src/main/python/ml/train_validation_split.py index 161a200c61b6..5f5c52aca8c4 100644 --- a/examples/src/main/python/ml/train_validation_split.py +++ b/examples/src/main/python/ml/train_validation_split.py @@ -15,13 +15,12 @@ # limitations under the License. # -from pyspark import SparkContext # $example on$ from pyspark.ml.evaluation import RegressionEvaluator from pyspark.ml.regression import LinearRegression from pyspark.ml.tuning import ParamGridBuilder, TrainValidationSplit -from pyspark.sql import SQLContext # $example off$ +from pyspark.sql import SparkSession """ This example demonstrates applying TrainValidationSplit to split data @@ -32,11 +31,13 @@ """ if __name__ == "__main__": - sc = SparkContext(appName="TrainValidationSplit") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("TrainValidationSplit")\ + .getOrCreate() # $example on$ # Prepare training and test data. - data = sqlContext.read.format("libsvm")\ + data = spark.read.format("libsvm")\ .load("data/mllib/sample_linear_regression_data.txt") train, test = data.randomSplit([0.7, 0.3]) lr = LinearRegression(maxIter=10, regParam=0.1) @@ -65,4 +66,4 @@ for row in prediction.take(5): print(row) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/vector_assembler_example.py b/examples/src/main/python/ml/vector_assembler_example.py index 04f64839f188..b955ff00a819 100644 --- a/examples/src/main/python/ml/vector_assembler_example.py +++ b/examples/src/main/python/ml/vector_assembler_example.py @@ -17,19 +17,20 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.mllib.linalg import Vectors from pyspark.ml.feature import VectorAssembler # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="VectorAssemblerExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("VectorAssemblerExample")\ + .getOrCreate() # $example on$ - dataset = sqlContext.createDataFrame( + dataset = spark.createDataFrame( [(0, 18, 1.0, Vectors.dense([0.0, 10.0, 0.5]), 1.0)], ["id", "hour", "mobile", "userFeatures", "clicked"]) assembler = VectorAssembler( @@ -39,4 +40,4 @@ print(output.select("features", "clicked").first()) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/vector_indexer_example.py b/examples/src/main/python/ml/vector_indexer_example.py index 146f41c1dd90..9b00e0f84136 100644 --- a/examples/src/main/python/ml/vector_indexer_example.py +++ b/examples/src/main/python/ml/vector_indexer_example.py @@ -17,18 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import VectorIndexer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="VectorIndexerExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("VectorIndexerExample")\ + .getOrCreate() # $example on$ - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") indexer = VectorIndexer(inputCol="features", outputCol="indexed", maxCategories=10) indexerModel = indexer.fit(data) @@ -37,4 +38,4 @@ indexedData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/vector_slicer_example.py b/examples/src/main/python/ml/vector_slicer_example.py index 31a753073c13..b833a894eb84 100644 --- a/examples/src/main/python/ml/vector_slicer_example.py +++ b/examples/src/main/python/ml/vector_slicer_example.py @@ -17,20 +17,21 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import VectorSlicer from pyspark.mllib.linalg import Vectors from pyspark.sql.types import Row # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="VectorSlicerExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("VectorSlicerExample")\ + .getOrCreate() # $example on$ - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ Row(userFeatures=Vectors.sparse(3, {0: -2.0, 1: 2.3}),), Row(userFeatures=Vectors.dense([-2.0, 2.3, 0.0]),)]) @@ -41,4 +42,4 @@ output.select("userFeatures", "features").show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/word2vec_example.py b/examples/src/main/python/ml/word2vec_example.py index 53c77feb1014..66500bee152f 100644 --- a/examples/src/main/python/ml/word2vec_example.py +++ b/examples/src/main/python/ml/word2vec_example.py @@ -17,19 +17,20 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import Word2Vec # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="Word2VecExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("Word2VecExample")\ + .getOrCreate() # $example on$ # Input data: Each row is a bag of words from a sentence or document. - documentDF = sqlContext.createDataFrame([ + documentDF = spark.createDataFrame([ ("Hi I heard about Spark".split(" "), ), ("I wish Java could use case classes".split(" "), ), ("Logistic regression models are neat".split(" "), ) @@ -42,4 +43,4 @@ print(feature) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/mllib/binary_classification_metrics_example.py b/examples/src/main/python/mllib/binary_classification_metrics_example.py index 4e7ea289b253..daf000e38dcd 100644 --- a/examples/src/main/python/mllib/binary_classification_metrics_example.py +++ b/examples/src/main/python/mllib/binary_classification_metrics_example.py @@ -18,20 +18,25 @@ Binary Classification Metrics Example. """ from __future__ import print_function -from pyspark import SparkContext, SQLContext +from pyspark.sql import SparkSession # $example on$ from pyspark.mllib.classification import LogisticRegressionWithLBFGS from pyspark.mllib.evaluation import BinaryClassificationMetrics -from pyspark.mllib.util import MLUtils +from pyspark.mllib.regression import LabeledPoint # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="BinaryClassificationMetricsExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("BinaryClassificationMetricsExample")\ + .getOrCreate() + # $example on$ # Several of the methods available in scala are currently missing from pyspark # Load training data in LIBSVM format - data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_binary_classification_data.txt") + data = spark\ + .read.format("libsvm").load("data/mllib/sample_binary_classification_data.txt")\ + .rdd.map(lambda row: LabeledPoint(row[0], row[1])) # Split data into training (60%) and test (40%) training, test = data.randomSplit([0.6, 0.4], seed=11L) @@ -52,3 +57,5 @@ # Area under ROC curve print("Area under ROC = %s" % metrics.areaUnderROC) # $example off$ + + spark.stop() diff --git a/examples/src/main/python/mllib/gaussian_mixture_model.py b/examples/src/main/python/mllib/gaussian_mixture_model.py index 69e836fc1d06..6b46e27ddaaa 100644 --- a/examples/src/main/python/mllib/gaussian_mixture_model.py +++ b/examples/src/main/python/mllib/gaussian_mixture_model.py @@ -20,6 +20,10 @@ """ from __future__ import print_function +import sys +if sys.version >= '3': + long = int + import random import argparse import numpy as np diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index 2c188759328f..ac7246938d3b 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -20,33 +20,31 @@ import os import sys -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession from pyspark.sql.types import Row, StructField, StructType, StringType, IntegerType if __name__ == "__main__": - sc = SparkContext(appName="PythonSQL") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("PythonSQL")\ + .getOrCreate() - # RDD is created from a list of rows - some_rdd = sc.parallelize([Row(name="John", age=19), - Row(name="Smith", age=23), - Row(name="Sarah", age=18)]) - # Infer schema from the first row, create a DataFrame and print the schema - some_df = sqlContext.createDataFrame(some_rdd) + # A list of Rows. Infer schema from the first row, create a DataFrame and print the schema + rows = [Row(name="John", age=19), Row(name="Smith", age=23), Row(name="Sarah", age=18)] + some_df = spark.createDataFrame(rows) some_df.printSchema() - # Another RDD is created from a list of tuples - another_rdd = sc.parallelize([("John", 19), ("Smith", 23), ("Sarah", 18)]) + # A list of tuples + tuples = [("John", 19), ("Smith", 23), ("Sarah", 18)] # Schema with two fields - person_name and person_age schema = StructType([StructField("person_name", StringType(), False), StructField("person_age", IntegerType(), False)]) # Create a DataFrame by applying the schema to the RDD and print the schema - another_df = sqlContext.createDataFrame(another_rdd, schema) + another_df = spark.createDataFrame(tuples, schema) another_df.printSchema() # root - # |-- age: integer (nullable = true) + # |-- age: long (nullable = true) # |-- name: string (nullable = true) # A JSON dataset is pointed to by path. @@ -57,7 +55,7 @@ else: path = sys.argv[1] # Create a DataFrame from the file(s) pointed to by path - people = sqlContext.jsonFile(path) + people = spark.read.json(path) # root # |-- person_name: string (nullable = false) # |-- person_age: integer (nullable = false) @@ -65,16 +63,16 @@ # The inferred schema can be visualized using the printSchema() method. people.printSchema() # root - # |-- age: IntegerType - # |-- name: StringType + # |-- age: long (nullable = true) + # |-- name: string (nullable = true) - # Register this DataFrame as a table. - people.registerAsTable("people") + # Creates a temporary view using the DataFrame. + people.createOrReplaceTempView("people") - # SQL statements can be run by using the sql methods provided by sqlContext - teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + # SQL statements can be run by using the sql methods provided by `spark` + teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") for each in teenagers.collect(): print(each[0]) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py index 1ba5e9fb7899..398ac8d2d8f5 100644 --- a/examples/src/main/python/streaming/sql_network_wordcount.py +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -33,13 +33,16 @@ from pyspark import SparkContext from pyspark.streaming import StreamingContext -from pyspark.sql import SQLContext, Row +from pyspark.sql import Row, SparkSession -def getSqlContextInstance(sparkContext): - if ('sqlContextSingletonInstance' not in globals()): - globals()['sqlContextSingletonInstance'] = SQLContext(sparkContext) - return globals()['sqlContextSingletonInstance'] +def getSparkSessionInstance(sparkConf): + if ('sparkSessionSingletonInstance' not in globals()): + globals()['sparkSessionSingletonInstance'] = SparkSession\ + .builder\ + .config(conf=sparkConf)\ + .getOrCreate() + return globals()['sparkSessionSingletonInstance'] if __name__ == "__main__": @@ -60,19 +63,19 @@ def process(time, rdd): print("========= %s =========" % str(time)) try: - # Get the singleton instance of SQLContext - sqlContext = getSqlContextInstance(rdd.context) + # Get the singleton instance of SparkSession + spark = getSparkSessionInstance(rdd.context.getConf()) # Convert RDD[String] to RDD[Row] to DataFrame rowRdd = rdd.map(lambda w: Row(word=w)) - wordsDataFrame = sqlContext.createDataFrame(rowRdd) + wordsDataFrame = spark.createDataFrame(rowRdd) - # Register as table - wordsDataFrame.registerTempTable("words") + # Creates a temporary view using the DataFrame. + wordsDataFrame.createOrReplaceTempView("words") # Do word count on table using SQL and print it wordCountsDataFrame = \ - sqlContext.sql("select word, count(*) as total from words group by word") + spark.sql("select word, count(*) as total from words group by word") wordCountsDataFrame.show() except: pass diff --git a/examples/src/main/r/data-manipulation.R b/examples/src/main/r/data-manipulation.R index 594bf49d6015..58a30135aa92 100644 --- a/examples/src/main/r/data-manipulation.R +++ b/examples/src/main/r/data-manipulation.R @@ -20,8 +20,7 @@ # The data set is made up of 227,496 rows x 14 columns. # To run this example use -# ./bin/sparkR --packages com.databricks:spark-csv_2.10:1.0.3 -# examples/src/main/r/data-manipulation.R +# ./bin/spark-submit examples/src/main/r/data-manipulation.R # Load SparkR library into your R session library(SparkR) @@ -29,7 +28,7 @@ library(SparkR) args <- commandArgs(trailing = TRUE) if (length(args) != 1) { - print("Usage: data-manipulation.R ") print("The data can be downloaded from: http://s3-us-west-2.amazonaws.com/sparkr-data/flights.csv") q("no") } @@ -53,7 +52,7 @@ SFO_df <- flights_df[flights_df$dest == "SFO", ] SFO_DF <- createDataFrame(sqlContext, SFO_df) # Directly create a SparkDataFrame from the source data -flightsDF <- read.df(sqlContext, flightsCsvPath, source = "com.databricks.spark.csv", header = "true") +flightsDF <- read.df(sqlContext, flightsCsvPath, source = "csv", header = "true") # Print the schema of this SparkDataFrame printSchema(flightsDF) diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index fa1010195551..97aefac025e5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -24,7 +24,7 @@ import org.apache.commons.math3.linear._ * Alternating least squares matrix factorization. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.recommendation.ALS + * please refer to org.apache.spark.ml.recommendation.ALS. */ object LocalALS { @@ -96,7 +96,7 @@ object LocalALS { def showWarning() { System.err.println( """WARN: This is a naive implementation of ALS and is given as an example! - |Please use the ALS method found in org.apache.spark.mllib.recommendation + |Please use org.apache.spark.ml.recommendation.ALS |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala index bec89f7c3dff..3d02ce05619a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala @@ -26,8 +26,7 @@ import breeze.linalg.{DenseVector, Vector} * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object LocalFileLR { val D = 10 // Number of dimensions @@ -43,8 +42,7 @@ object LocalFileLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala index f8961847f3df..fca585c2a362 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala @@ -29,7 +29,7 @@ import breeze.linalg.{squaredDistance, DenseVector, Vector} * K-means clustering. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.clustering.KMeans + * please refer to org.apache.spark.ml.clustering.KMeans. */ object LocalKMeans { val N = 1000 @@ -66,7 +66,7 @@ object LocalKMeans { def showWarning() { System.err.println( """WARN: This is a naive implementation of KMeans Clustering and is given as an example! - |Please use the KMeans method found in org.apache.spark.mllib.clustering + |Please use org.apache.spark.ml.clustering.KMeans |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala index 0baf6db607ad..13ccc2ae7c3d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala @@ -26,8 +26,7 @@ import breeze.linalg.{DenseVector, Vector} * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object LocalLR { val N = 10000 // Number of data points @@ -50,8 +49,7 @@ object LocalLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index 4263680c6fde..b06c62980200 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -26,7 +26,7 @@ import org.apache.spark._ * Alternating least squares matrix factorization. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.recommendation.ALS + * please refer to org.apache.spark.ml.recommendation.ALS. */ object SparkALS { @@ -81,7 +81,7 @@ object SparkALS { def showWarning() { System.err.println( """WARN: This is a naive implementation of ALS and is given as an example! - |Please use the ALS method found in org.apache.spark.mllib.recommendation + |Please use org.apache.spark.ml.recommendation.ALS |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 7463b868ff19..c514eb0fa51a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -31,8 +31,7 @@ import org.apache.spark._ * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object SparkHdfsLR { val D = 10 // Number of dimensions @@ -54,8 +53,7 @@ object SparkHdfsLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index d9f94a42b1a0..676164806e19 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -26,7 +26,7 @@ import org.apache.spark.{SparkConf, SparkContext} * K-means clustering. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.clustering.KMeans + * please refer to org.apache.spark.ml.clustering.KMeans. */ object SparkKMeans { @@ -52,7 +52,7 @@ object SparkKMeans { def showWarning() { System.err.println( """WARN: This is a naive implementation of KMeans Clustering and is given as an example! - |Please use the KMeans method found in org.apache.spark.mllib.clustering + |Please use org.apache.spark.ml.clustering.KMeans |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index acd8656b65a6..718f84f6450e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -31,8 +31,7 @@ import org.apache.spark._ * Usage: SparkLR [slices] * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object SparkLR { val N = 10000 // Number of data points @@ -55,8 +54,7 @@ object SparkLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala index 21f58ddf3cfb..b6d7b369162d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala @@ -18,25 +18,29 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.regression.AFTSurvivalRegression -import org.apache.spark.mllib.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** - * An example for AFTSurvivalRegression. + * An example demonstrating AFTSurvivalRegression. + * Run with + * {{{ + * bin/run-example ml.AFTSurvivalRegressionExample + * }}} */ object AFTSurvivalRegressionExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("AFTSurvivalRegressionExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("AFTSurvivalRegressionExample") + .getOrCreate() // $example on$ - val training = sqlContext.createDataFrame(Seq( + val training = spark.createDataFrame(Seq( (1.218, 1.0, Vectors.dense(1.560, -0.605)), (2.949, 0.0, Vectors.dense(0.346, 2.158)), (3.627, 0.0, Vectors.dense(1.380, 0.231)), @@ -56,7 +60,7 @@ object AFTSurvivalRegressionExample { model.transform(training).show(false) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala index a79e15c767e1..da19ea9f10ec 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala @@ -18,39 +18,40 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.evaluation.RegressionEvaluator import org.apache.spark.ml.recommendation.ALS // $example off$ -import org.apache.spark.sql.SQLContext -// $example on$ -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.DoubleType -// $example off$ +import org.apache.spark.sql.SparkSession +/** + * An example demonstrating ALS. + * Run with + * {{{ + * bin/run-example ml.ALSExample + * }}} + */ object ALSExample { // $example on$ case class Rating(userId: Int, movieId: Int, rating: Float, timestamp: Long) - object Rating { - def parseRating(str: String): Rating = { - val fields = str.split("::") - assert(fields.size == 4) - Rating(fields(0).toInt, fields(1).toInt, fields(2).toFloat, fields(3).toLong) - } + def parseRating(str: String): Rating = { + val fields = str.split("::") + assert(fields.size == 4) + Rating(fields(0).toInt, fields(1).toInt, fields(2).toFloat, fields(3).toLong) } // $example off$ def main(args: Array[String]) { - val conf = new SparkConf().setAppName("ALSExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession + .builder + .appName("ALSExample") + .getOrCreate() + import spark.implicits._ // $example on$ - val ratings = sc.textFile("data/mllib/als/sample_movielens_ratings.txt") - .map(Rating.parseRating) + val ratings = spark.read.text("data/mllib/als/sample_movielens_ratings.txt") + .map(parseRating) .toDF() val Array(training, test) = ratings.randomSplit(Array(0.8, 0.2)) @@ -65,8 +66,6 @@ object ALSExample { // Evaluate the model by computing the RMSE on the test data val predictions = model.transform(test) - .withColumn("rating", col("rating").cast(DoubleType)) - .withColumn("prediction", col("prediction").cast(DoubleType)) val evaluator = new RegressionEvaluator() .setMetricName("rmse") @@ -75,7 +74,8 @@ object ALSExample { val rmse = evaluator.evaluate(predictions) println(s"Root-mean-square error = $rmse") // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala index 2ed8101c133c..82bc14789b46 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala @@ -18,20 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.Binarizer // $example off$ -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, SparkSession} object BinarizerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("BinarizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("BinarizerExample") + .getOrCreate() // $example on$ val data = Array((0, 0.1), (1, 0.8), (2, 0.2)) - val dataFrame: DataFrame = sqlContext.createDataFrame(data).toDF("label", "feature") + val dataFrame: DataFrame = spark.createDataFrame(data).toDF("label", "feature") val binarizer: Binarizer = new Binarizer() .setInputCol("feature") @@ -42,7 +42,8 @@ object BinarizerExample { val binarizedFeatures = binarizedDataFrame.select("binarized_feature") binarizedFeatures.collect().foreach(println) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala new file mode 100644 index 000000000000..5f8f2c99cbaf --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala @@ -0,0 +1,65 @@ +/* + * 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.examples.ml + +// scalastyle:off println + +// $example on$ +import org.apache.spark.ml.clustering.BisectingKMeans +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating bisecting k-means clustering. + * Run with + * {{{ + * bin/run-example ml.BisectingKMeansExample + * }}} + */ +object BisectingKMeansExample { + + def main(args: Array[String]): Unit = { + // Creates a SparkSession + val spark = SparkSession + .builder + .appName("BisectingKMeansExample") + .getOrCreate() + + // $example on$ + // Loads data. + val dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") + + // Trains a bisecting k-means model. + val bkm = new BisectingKMeans().setK(2).setSeed(1) + val model = bkm.fit(dataset) + + // Evaluate clustering. + val cost = model.computeCost(dataset) + println(s"Within Set Sum of Squared Errors = $cost") + + // Shows the result. + println("Cluster Centers: ") + val centers = model.clusterCenters + centers.foreach(println) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println + diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala index 6f6236a2b058..38cce34bb509 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala @@ -18,23 +18,23 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.Bucketizer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object BucketizerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("BucketizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("BucketizerExample") + .getOrCreate() // $example on$ val splits = Array(Double.NegativeInfinity, -0.5, 0.0, 0.5, Double.PositiveInfinity) val data = Array(-0.5, -0.3, 0.0, 0.2) - val dataFrame = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") + val dataFrame = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") val bucketizer = new Bucketizer() .setInputCol("features") @@ -45,7 +45,7 @@ object BucketizerExample { val bucketedData = bucketizer.transform(dataFrame) bucketedData.show() // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala index 2be61537e613..c9394dd9c64b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala @@ -18,20 +18,19 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.ChiSqSelector -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object ChiSqSelectorExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("ChiSqSelectorExample") - val sc = new SparkContext(conf) - - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession + .builder + .appName("ChiSqSelectorExample") + .getOrCreate() + import spark.implicits._ // $example on$ val data = Seq( @@ -40,7 +39,7 @@ object ChiSqSelectorExample { (9, Vectors.dense(1.0, 0.0, 15.0, 0.1), 0.0) ) - val df = sc.parallelize(data).toDF("id", "features", "clicked") + val df = spark.createDataset(data).toDF("id", "features", "clicked") val selector = new ChiSqSelector() .setNumTopFeatures(1) @@ -51,7 +50,7 @@ object ChiSqSelectorExample { val result = selector.fit(df).transform(df) result.show() // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala index 7d07fc7dd113..51aa5179fa4a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala @@ -18,20 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.{CountVectorizer, CountVectorizerModel} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object CountVectorizerExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("CounterVectorizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("CounterVectorizerExample") + .getOrCreate() // $example on$ - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, Array("a", "b", "c")), (1, Array("a", "b", "b", "c", "a")) )).toDF("id", "words") @@ -51,6 +51,8 @@ object CountVectorizerExample { cvModel.transform(df).select("features").show() // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala index dc26b55a768a..ddc671752872 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala @@ -18,18 +18,18 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.DCT -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object DCTExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("DCTExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("DCTExample") + .getOrCreate() // $example on$ val data = Seq( @@ -37,7 +37,7 @@ object DCTExample { Vectors.dense(-1.0, 2.0, 4.0, -7.0), Vectors.dense(14.0, -2.0, -5.0, 1.0)) - val df = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") + val df = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") val dct = new DCT() .setInputCol("features") @@ -47,7 +47,8 @@ object DCTExample { val dctDf = dct.transform(df) dctDf.select("featuresDCT").show(3) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala index 7e608a281203..c69027babba8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala @@ -23,11 +23,11 @@ import java.io.File import com.google.common.io.Files import scopt.OptionParser -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.examples.mllib.AbstractParams -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Row, SparkSession} /** * An example of how to use [[org.apache.spark.sql.DataFrame]] for ML. Run with @@ -62,14 +62,14 @@ object DataFrameExample { } def run(params: Params) { - - val conf = new SparkConf().setAppName(s"DataFrameExample with $params") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName(s"DataFrameExample with $params") + .getOrCreate() // Load input data println(s"Loading LIBSVM file with UDT from ${params.input}.") - val df: DataFrame = sqlContext.read.format("libsvm").load(params.input).cache() + val df: DataFrame = spark.read.format("libsvm").load(params.input).cache() println("Schema from LIBSVM:") df.printSchema() println(s"Loaded training data as a DataFrame with ${df.count()} records.") @@ -81,7 +81,7 @@ object DataFrameExample { // Convert features column to an RDD of vectors. val features = df.select("features").rdd.map { case Row(v: Vector) => v } val featureSummary = features.aggregate(new MultivariateOnlineSummarizer())( - (summary, feat) => summary.add(feat), + (summary, feat) => summary.add(Vectors.fromML(feat)), (sum1, sum2) => sum1.merge(sum2)) println(s"Selected features column with average values:\n ${featureSummary.mean.toString}") @@ -94,11 +94,11 @@ object DataFrameExample { // Load the records back. println(s"Loading Parquet file with UDT from $outputDir.") - val newDF = sqlContext.read.parquet(outputDir) + val newDF = spark.read.parquet(outputDir) println(s"Schema from Parquet:") newDF.printSchema() - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala index 224d8da5f0ec..b3103ced9145 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala @@ -18,7 +18,6 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.DecisionTreeClassificationModel @@ -26,16 +25,17 @@ import org.apache.spark.ml.classification.DecisionTreeClassifier import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator import org.apache.spark.ml.feature.{IndexToString, StringIndexer, VectorIndexer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object DecisionTreeClassificationExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("DecisionTreeClassificationExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("DecisionTreeClassificationExample") + .getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Index labels, adding metadata to the label column. // Fit on whole dataset to include all labels in index. @@ -47,10 +47,10 @@ object DecisionTreeClassificationExample { val featureIndexer = new VectorIndexer() .setInputCol("features") .setOutputCol("indexedFeatures") - .setMaxCategories(4) // features with > 4 distinct values are treated as continuous + .setMaxCategories(4) // features with > 4 distinct values are treated as continuous. .fit(data) - // Split the data into training and test sets (30% held out for testing) + // Split the data into training and test sets (30% held out for testing). val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) // Train a DecisionTree model. @@ -64,11 +64,11 @@ object DecisionTreeClassificationExample { .setOutputCol("predictedLabel") .setLabels(labelIndexer.labels) - // Chain indexers and tree in a Pipeline + // Chain indexers and tree in a Pipeline. val pipeline = new Pipeline() .setStages(Array(labelIndexer, featureIndexer, dt, labelConverter)) - // Train model. This also runs the indexers. + // Train model. This also runs the indexers. val model = pipeline.fit(trainingData) // Make predictions. @@ -77,7 +77,7 @@ object DecisionTreeClassificationExample { // Select example rows to display. predictions.select("predictedLabel", "label", "features").show(5) - // Select (prediction, true label) and compute test error + // Select (prediction, true label) and compute test error. val evaluator = new MulticlassClassificationEvaluator() .setLabelCol("indexedLabel") .setPredictionCol("prediction") @@ -88,6 +88,8 @@ object DecisionTreeClassificationExample { val treeModel = model.stages(2).asInstanceOf[DecisionTreeClassificationModel] println("Learned classification tree model:\n" + treeModel.toDebugString) // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index d2560cc00ba0..5e51dbad760f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -23,24 +23,23 @@ import scala.language.reflectiveCalls import scopt.OptionParser -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.{Pipeline, PipelineStage, Transformer} import org.apache.spark.ml.classification.{DecisionTreeClassificationModel, DecisionTreeClassifier} import org.apache.spark.ml.feature.{StringIndexer, VectorIndexer} +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.regression.{DecisionTreeRegressionModel, DecisionTreeRegressor} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.evaluation.{MulticlassMetrics, RegressionMetrics} -import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, SparkSession} /** * An example runner for decision trees. Run with * {{{ * ./bin/run-example ml.DecisionTreeExample [options] * }}} - * Note that Decision Trees can take a large amount of memory. If the run-example command above + * Note that Decision Trees can take a large amount of memory. If the run-example command above * fails, try running via spark-submit and specifying the amount of memory as at least 1g. * For local mode, run * {{{ @@ -87,7 +86,7 @@ object DecisionTreeExample { .text(s"min info gain required to create a split, default: ${defaultParams.minInfoGain}") .action((x, c) => c.copy(minInfoGain = x)) opt[Double]("fracTest") - .text(s"fraction of data to hold out for testing. If given option testInput, " + + .text(s"fraction of data to hold out for testing. If given option testInput, " + s"this option is ignored. default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) opt[Boolean]("cacheNodeIds") @@ -106,7 +105,7 @@ object DecisionTreeExample { s"default: ${defaultParams.checkpointInterval}") .action((x, c) => c.copy(checkpointInterval = x)) opt[String]("testInput") - .text(s"input path to test dataset. If given, option fracTest is ignored." + + .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") .action((x, c) => c.copy(testInput = x)) opt[String]("dataFormat") @@ -134,18 +133,18 @@ object DecisionTreeExample { /** Load a dataset from the given path, using the given format */ private[ml] def loadData( - sqlContext: SQLContext, + spark: SparkSession, path: String, format: String, expectedNumFeatures: Option[Int] = None): DataFrame = { - import sqlContext.implicits._ + import spark.implicits._ format match { - case "dense" => MLUtils.loadLabeledPoints(sqlContext.sparkContext, path).toDF() + case "dense" => MLUtils.loadLabeledPoints(spark.sparkContext, path).toDF() case "libsvm" => expectedNumFeatures match { - case Some(numFeatures) => sqlContext.read.option("numFeatures", numFeatures.toString) + case Some(numFeatures) => spark.read.option("numFeatures", numFeatures.toString) .format("libsvm").load(path) - case None => sqlContext.read.format("libsvm").load(path) + case None => spark.read.format("libsvm").load(path) } case _ => throw new IllegalArgumentException(s"Bad data format: $format") } @@ -157,27 +156,28 @@ object DecisionTreeExample { * @param dataFormat "libsvm" or "dense" * @param testInput Path to test dataset. * @param algo Classification or Regression - * @param fracTest Fraction of input data to hold out for testing. Ignored if testInput given. + * @param fracTest Fraction of input data to hold out for testing. Ignored if testInput given. * @return (training dataset, test dataset) */ private[ml] def loadDatasets( - sc: SparkContext, input: String, dataFormat: String, testInput: String, algo: String, fracTest: Double): (DataFrame, DataFrame) = { - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .getOrCreate() // Load training data - val origExamples: DataFrame = loadData(sqlContext, input, dataFormat) + val origExamples: DataFrame = loadData(spark, input, dataFormat) // Load or create test set val dataframes: Array[DataFrame] = if (testInput != "") { // Load testInput. val numFeatures = origExamples.first().getAs[Vector](1).size val origTestExamples: DataFrame = - loadData(sqlContext, testInput, dataFormat, Some(numFeatures)) + loadData(spark, testInput, dataFormat, Some(numFeatures)) Array(origExamples, origTestExamples) } else { // Split input into training, test. @@ -198,18 +198,21 @@ object DecisionTreeExample { } def run(params: Params) { - val conf = new SparkConf().setAppName(s"DecisionTreeExample with $params") - val sc = new SparkContext(conf) - params.checkpointDir.foreach(sc.setCheckpointDir) + val spark = SparkSession + .builder + .appName(s"DecisionTreeExample with $params") + .getOrCreate() + + params.checkpointDir.foreach(spark.sparkContext.setCheckpointDir) val algo = params.algo.toLowerCase println(s"DecisionTreeExample with parameters:\n$params") // Load training and test data and cache it. val (training: DataFrame, test: DataFrame) = - loadDatasets(sc, params.input, params.dataFormat, params.testInput, algo, params.fracTest) + loadDatasets(params.input, params.dataFormat, params.testInput, algo, params.fracTest) - // Set up Pipeline + // Set up Pipeline. val stages = new mutable.ArrayBuffer[PipelineStage]() // (1) For classification, re-index classes. val labelColName = if (algo == "classification") "indexedLabel" else "label" @@ -226,7 +229,7 @@ object DecisionTreeExample { .setOutputCol("indexedFeatures") .setMaxCategories(10) stages += featuresIndexer - // (3) Learn Decision Tree + // (3) Learn Decision Tree. val dt = algo match { case "classification" => new DecisionTreeClassifier() @@ -253,13 +256,13 @@ object DecisionTreeExample { stages += dt val pipeline = new Pipeline().setStages(stages.toArray) - // Fit the Pipeline + // Fit the Pipeline. val startTime = System.nanoTime() val pipelineModel = pipeline.fit(training) val elapsedTime = (System.nanoTime() - startTime) / 1e9 println(s"Training time: $elapsedTime seconds") - // Get the trained Decision Tree from the fitted PipelineModel + // Get the trained Decision Tree from the fitted PipelineModel. algo match { case "classification" => val treeModel = pipelineModel.stages.last.asInstanceOf[DecisionTreeClassificationModel] @@ -278,7 +281,7 @@ object DecisionTreeExample { case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") } - // Evaluate model on training, test data + // Evaluate model on training, test data. algo match { case "classification" => println("Training data results:") @@ -294,11 +297,11 @@ object DecisionTreeExample { throw new IllegalArgumentException("Algo ${params.algo} not supported.") } - sc.stop() + spark.stop() } /** - * Evaluate the given ClassificationModel on data. Print the results. + * Evaluate the given ClassificationModel on data. Print the results. * @param model Must fit ClassificationModel abstraction * @param data DataFrame with "prediction" and labelColName columns * @param labelColName Name of the labelCol parameter for the model @@ -312,7 +315,7 @@ object DecisionTreeExample { val fullPredictions = model.transform(data).cache() val predictions = fullPredictions.select("prediction").rdd.map(_.getDouble(0)) val labels = fullPredictions.select(labelColName).rdd.map(_.getDouble(0)) - // Print number of classes for reference + // Print number of classes for reference. val numClasses = MetadataUtils.getNumClasses(fullPredictions.schema(labelColName)) match { case Some(n) => n case None => throw new RuntimeException( @@ -323,7 +326,7 @@ object DecisionTreeExample { } /** - * Evaluate the given RegressionModel on data. Print the results. + * Evaluate the given RegressionModel on data. Print the results. * @param model Must fit RegressionModel abstraction * @param data DataFrame with "prediction" and labelColName columns * @param labelColName Name of the labelCol parameter for the model diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala index ad32e5635a3e..ee61200ad1d0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala @@ -18,7 +18,6 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.evaluation.RegressionEvaluator @@ -26,17 +25,18 @@ import org.apache.spark.ml.feature.VectorIndexer import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.regression.DecisionTreeRegressor // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object DecisionTreeRegressionExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("DecisionTreeRegressionExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("DecisionTreeRegressionExample") + .getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Automatically identify categorical features, and index them. // Here, we treat features with > 4 distinct values as continuous. @@ -46,7 +46,7 @@ object DecisionTreeRegressionExample { .setMaxCategories(4) .fit(data) - // Split the data into training and test sets (30% held out for testing) + // Split the data into training and test sets (30% held out for testing). val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) // Train a DecisionTree model. @@ -54,11 +54,11 @@ object DecisionTreeRegressionExample { .setLabelCol("label") .setFeaturesCol("indexedFeatures") - // Chain indexer and tree in a Pipeline + // Chain indexer and tree in a Pipeline. val pipeline = new Pipeline() .setStages(Array(featureIndexer, dt)) - // Train model. This also runs the indexer. + // Train model. This also runs the indexer. val model = pipeline.fit(trainingData) // Make predictions. @@ -67,7 +67,7 @@ object DecisionTreeRegressionExample { // Select example rows to display. predictions.select("prediction", "label", "features").show(5) - // Select (prediction, true label) and compute test error + // Select (prediction, true label) and compute test error. val evaluator = new RegressionEvaluator() .setLabelCol("label") .setPredictionCol("prediction") @@ -78,6 +78,8 @@ object DecisionTreeRegressionExample { val treeModel = model.stages(1).asInstanceOf[DecisionTreeRegressionModel] println("Learned regression tree model:\n" + treeModel.toDebugString) // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index 8d127f9b3542..d94d837d10e9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -18,13 +18,12 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.classification.{ClassificationModel, Classifier, ClassifierParams} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} import org.apache.spark.ml.param.{IntParam, ParamMap} import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext} +import org.apache.spark.sql.{Dataset, Row, SparkSession} /** * A simple example demonstrating how to write your own learning algorithm using Estimator, @@ -38,19 +37,20 @@ import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext} object DeveloperApiExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("DeveloperApiExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession + .builder + .appName("DeveloperApiExample") + .getOrCreate() + import spark.implicits._ // Prepare training data. - val training = sc.parallelize(Seq( + val training = spark.createDataFrame(Seq( LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)))) - // Create a LogisticRegression instance. This instance is an Estimator. + // Create a LogisticRegression instance. This instance is an Estimator. val lr = new MyLogisticRegression() // Print out the parameters, documentation, and any default values. println("MyLogisticRegression parameters:\n" + lr.explainParams() + "\n") @@ -58,17 +58,17 @@ object DeveloperApiExample { // We may set parameters using setter methods. lr.setMaxIter(10) - // Learn a LogisticRegression model. This uses the parameters stored in lr. + // Learn a LogisticRegression model. This uses the parameters stored in lr. val model = lr.fit(training.toDF()) // Prepare test data. - val test = sc.parallelize(Seq( + val test = spark.createDataFrame(Seq( LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) // Make predictions on test data. - val sumPredictions: Double = model.transform(test.toDF()) + val sumPredictions: Double = model.transform(test) .select("features", "label", "prediction") .collect() .map { case Row(features: Vector, label: Double, prediction: Double) => @@ -77,14 +77,14 @@ object DeveloperApiExample { assert(sumPredictions == 0.0, "MyLogisticRegression predicted something other than 0, even though all coefficients are 0!") - sc.stop() + spark.stop() } } /** * Example of defining a parameter trait for a user-defined type of [[Classifier]]. * - * NOTE: This is private since it is an example. In practice, you may not want it to be private. + * NOTE: This is private since it is an example. In practice, you may not want it to be private. */ private trait MyLogisticRegressionParams extends ClassifierParams { @@ -96,7 +96,7 @@ private trait MyLogisticRegressionParams extends ClassifierParams { * - def getMyParamName * - def setMyParamName * Here, we have a trait to be mixed in with the Estimator and Model (MyLogisticRegression - * and MyLogisticRegressionModel). We place the setter (setMaxIter) method in the Estimator + * and MyLogisticRegressionModel). We place the setter (setMaxIter) method in the Estimator * class since the maxIter parameter is only used during training (not in the Model). */ val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") @@ -106,7 +106,7 @@ private trait MyLogisticRegressionParams extends ClassifierParams { /** * Example of defining a type of [[Classifier]]. * - * NOTE: This is private since it is an example. In practice, you may not want it to be private. + * NOTE: This is private since it is an example. In practice, you may not want it to be private. */ private class MyLogisticRegression(override val uid: String) extends Classifier[Vector, MyLogisticRegression, MyLogisticRegressionModel] @@ -138,7 +138,7 @@ private class MyLogisticRegression(override val uid: String) /** * Example of defining a type of [[ClassificationModel]]. * - * NOTE: This is private since it is an example. In practice, you may not want it to be private. + * NOTE: This is private since it is an example. In practice, you may not want it to be private. */ private class MyLogisticRegressionModel( override val uid: String, @@ -169,7 +169,7 @@ private class MyLogisticRegressionModel( Vectors.dense(-margin, margin) } - /** Number of classes the label can take. 2 indicates binary classification. */ + /** Number of classes the label can take. 2 indicates binary classification. */ override val numClasses: Int = 2 /** Number of features the model was trained on. */ diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala index 629d322c4357..c0ffc01934b6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala @@ -18,22 +18,22 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.ElementwiseProduct -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object ElementwiseProductExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("ElementwiseProductExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("ElementwiseProductExample") + .getOrCreate() // $example on$ // Create some vector data; also works for sparse vectors - val dataFrame = sqlContext.createDataFrame(Seq( + val dataFrame = spark.createDataFrame(Seq( ("a", Vectors.dense(1.0, 2.0, 3.0)), ("b", Vectors.dense(4.0, 5.0, 6.0)))).toDF("id", "vector") @@ -46,7 +46,8 @@ object ElementwiseProductExample { // Batch transform the vectors to create new column: transformer.transform(dataFrame).show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala index 65e3c365abb3..f18d86e1a692 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala @@ -18,32 +18,32 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.sql.Row // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object EstimatorTransformerParamExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("EstimatorTransformerParamExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("EstimatorTransformerParamExample") + .getOrCreate() // $example on$ // Prepare training data from a list of (label, features) tuples. - val training = sqlContext.createDataFrame(Seq( + val training = spark.createDataFrame(Seq( (1.0, Vectors.dense(0.0, 1.1, 0.1)), (0.0, Vectors.dense(2.0, 1.0, -1.0)), (0.0, Vectors.dense(2.0, 1.3, 1.0)), (1.0, Vectors.dense(0.0, 1.2, -0.5)) )).toDF("label", "features") - // Create a LogisticRegression instance. This instance is an Estimator. + // Create a LogisticRegression instance. This instance is an Estimator. val lr = new LogisticRegression() // Print out the parameters, documentation, and any default values. println("LogisticRegression parameters:\n" + lr.explainParams() + "\n") @@ -52,7 +52,7 @@ object EstimatorTransformerParamExample { lr.setMaxIter(10) .setRegParam(0.01) - // Learn a LogisticRegression model. This uses the parameters stored in lr. + // Learn a LogisticRegression model. This uses the parameters stored in lr. val model1 = lr.fit(training) // Since model1 is a Model (i.e., a Transformer produced by an Estimator), // we can view the parameters it used during fit(). @@ -63,11 +63,11 @@ object EstimatorTransformerParamExample { // We may alternatively specify parameters using a ParamMap, // which supports several methods for specifying parameters. val paramMap = ParamMap(lr.maxIter -> 20) - .put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. + .put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. .put(lr.regParam -> 0.1, lr.threshold -> 0.55) // Specify multiple Params. // One can also combine ParamMaps. - val paramMap2 = ParamMap(lr.probabilityCol -> "myProbability") // Change output column name + val paramMap2 = ParamMap(lr.probabilityCol -> "myProbability") // Change output column name. val paramMapCombined = paramMap ++ paramMap2 // Now learn a new model using the paramMapCombined parameters. @@ -76,7 +76,7 @@ object EstimatorTransformerParamExample { println("Model 2 was fit using parameters: " + model2.parent.extractParamMap) // Prepare test data. - val test = sqlContext.createDataFrame(Seq( + val test = spark.createDataFrame(Seq( (1.0, Vectors.dense(-1.0, 1.5, 1.3)), (0.0, Vectors.dense(3.0, 2.0, -0.1)), (1.0, Vectors.dense(0.0, 2.2, -1.5)) @@ -94,7 +94,7 @@ object EstimatorTransformerParamExample { } // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala index 6b0be0f34e19..a4274ae95405 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala @@ -23,13 +23,12 @@ import scala.language.reflectiveCalls import scopt.OptionParser -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.{Pipeline, PipelineStage} import org.apache.spark.ml.classification.{GBTClassificationModel, GBTClassifier} import org.apache.spark.ml.feature.{StringIndexer, VectorIndexer} import org.apache.spark.ml.regression.{GBTRegressionModel, GBTRegressor} -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, SparkSession} /** @@ -37,7 +36,7 @@ import org.apache.spark.sql.DataFrame * {{{ * ./bin/run-example ml.GBTExample [options] * }}} - * Decision Trees and ensembles can take a large amount of memory. If the run-example command + * Decision Trees and ensembles can take a large amount of memory. If the run-example command * above fails, try running via spark-submit and specifying the amount of memory as at least 1g. * For local mode, run * {{{ @@ -88,7 +87,7 @@ object GBTExample { .text(s"number of trees in ensemble, default: ${defaultParams.maxIter}") .action((x, c) => c.copy(maxIter = x)) opt[Double]("fracTest") - .text(s"fraction of data to hold out for testing. If given option testInput, " + + .text(s"fraction of data to hold out for testing. If given option testInput, " + s"this option is ignored. default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) opt[Boolean]("cacheNodeIds") @@ -109,7 +108,7 @@ object GBTExample { s"default: ${defaultParams.checkpointInterval}") .action((x, c) => c.copy(checkpointInterval = x)) opt[String]("testInput") - .text(s"input path to test dataset. If given, option fracTest is ignored." + + .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") .action((x, c) => c.copy(testInput = x)) opt[String]("dataFormat") @@ -136,15 +135,18 @@ object GBTExample { } def run(params: Params) { - val conf = new SparkConf().setAppName(s"GBTExample with $params") - val sc = new SparkContext(conf) - params.checkpointDir.foreach(sc.setCheckpointDir) + val spark = SparkSession + .builder + .appName(s"GBTExample with $params") + .getOrCreate() + + params.checkpointDir.foreach(spark.sparkContext.setCheckpointDir) val algo = params.algo.toLowerCase println(s"GBTExample with parameters:\n$params") // Load training and test data and cache it. - val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(sc, params.input, + val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(params.input, params.dataFormat, params.testInput, algo, params.fracTest) // Set up Pipeline @@ -164,7 +166,7 @@ object GBTExample { .setOutputCol("indexedFeatures") .setMaxCategories(10) stages += featuresIndexer - // (3) Learn GBT + // (3) Learn GBT. val dt = algo match { case "classification" => new GBTClassifier() @@ -193,13 +195,13 @@ object GBTExample { stages += dt val pipeline = new Pipeline().setStages(stages.toArray) - // Fit the Pipeline + // Fit the Pipeline. val startTime = System.nanoTime() val pipelineModel = pipeline.fit(training) val elapsedTime = (System.nanoTime() - startTime) / 1e9 println(s"Training time: $elapsedTime seconds") - // Get the trained GBT from the fitted PipelineModel + // Get the trained GBT from the fitted PipelineModel. algo match { case "classification" => val rfModel = pipelineModel.stages.last.asInstanceOf[GBTClassificationModel] @@ -218,7 +220,7 @@ object GBTExample { case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") } - // Evaluate model on training, test data + // Evaluate model on training, test data. algo match { case "classification" => println("Training data results:") @@ -234,7 +236,7 @@ object GBTExample { throw new IllegalArgumentException("Algo ${params.algo} not supported.") } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala new file mode 100644 index 000000000000..c484ee55569b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala @@ -0,0 +1,58 @@ +/* + * 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.examples.ml + +// scalastyle:off println + +// $example on$ +import org.apache.spark.ml.clustering.GaussianMixture +import org.apache.spark.sql.SparkSession +// $example off$ + +/** + * An example demonstrating Gaussian Mixture Model (GMM). + * Run with + * {{{ + * bin/run-example ml.GaussianMixtureExample + * }}} + */ +object GaussianMixtureExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession + val spark = SparkSession.builder.appName(s"${this.getClass.getSimpleName}").getOrCreate() + + // $example on$ + // Loads data + val dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") + + // Trains Gaussian Mixture Model + val gmm = new GaussianMixture() + .setK(2) + val model = gmm.fit(dataset) + + // output parameters of mixture model model + for (i <- 0 until model.getK) { + println("weight=%f\nmu=%s\nsigma=\n%s\n" format + (model.weights(i), model.gaussians(i).mean, model.gaussians(i).cov)) + } + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala new file mode 100644 index 000000000000..1b86d7cad0b3 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala @@ -0,0 +1,78 @@ +/* + * 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. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.regression.GeneralizedLinearRegression +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating generalized linear regression. + * Run with + * {{{ + * bin/run-example ml.GeneralizedLinearRegressionExample + * }}} + */ + +object GeneralizedLinearRegressionExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("GeneralizedLinearRegressionExample") + .getOrCreate() + + // $example on$ + // Load training data + val dataset = spark.read.format("libsvm") + .load("data/mllib/sample_linear_regression_data.txt") + + val glr = new GeneralizedLinearRegression() + .setFamily("gaussian") + .setLink("identity") + .setMaxIter(10) + .setRegParam(0.3) + + // Fit the model + val model = glr.fit(dataset) + + // Print the coefficients and intercept for generalized linear regression model + println(s"Coefficients: ${model.coefficients}") + println(s"Intercept: ${model.intercept}") + + // Summarize the model over the training set and print out some metrics + val summary = model.summary + println(s"Coefficient Standard Errors: ${summary.coefficientStandardErrors.mkString(",")}") + println(s"T Values: ${summary.tValues.mkString(",")}") + println(s"P Values: ${summary.pValues.mkString(",")}") + println(s"Dispersion: ${summary.dispersion}") + println(s"Null Deviance: ${summary.nullDeviance}") + println(s"Residual Degree Of Freedom Null: ${summary.residualDegreeOfFreedomNull}") + println(s"Deviance: ${summary.deviance}") + println(s"Residual Degree Of Freedom: ${summary.residualDegreeOfFreedom}") + println(s"AIC: ${summary.aic}") + println("Deviance Residuals: ") + summary.residuals().show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala index cd62a803820c..0d1ffbe2259c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala @@ -18,24 +18,24 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.{GBTClassificationModel, GBTClassifier} import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator import org.apache.spark.ml.feature.{IndexToString, StringIndexer, VectorIndexer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object GradientBoostedTreeClassifierExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("GradientBoostedTreeClassifierExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("GradientBoostedTreeClassifierExample") + .getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Index labels, adding metadata to the label column. // Fit on whole dataset to include all labels in index. @@ -51,7 +51,7 @@ object GradientBoostedTreeClassifierExample { .setMaxCategories(4) .fit(data) - // Split the data into training and test sets (30% held out for testing) + // Split the data into training and test sets (30% held out for testing). val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) // Train a GBT model. @@ -66,11 +66,11 @@ object GradientBoostedTreeClassifierExample { .setOutputCol("predictedLabel") .setLabels(labelIndexer.labels) - // Chain indexers and GBT in a Pipeline + // Chain indexers and GBT in a Pipeline. val pipeline = new Pipeline() .setStages(Array(labelIndexer, featureIndexer, gbt, labelConverter)) - // Train model. This also runs the indexers. + // Train model. This also runs the indexers. val model = pipeline.fit(trainingData) // Make predictions. @@ -79,7 +79,7 @@ object GradientBoostedTreeClassifierExample { // Select example rows to display. predictions.select("predictedLabel", "label", "features").show(5) - // Select (prediction, true label) and compute test error + // Select (prediction, true label) and compute test error. val evaluator = new MulticlassClassificationEvaluator() .setLabelCol("indexedLabel") .setPredictionCol("prediction") @@ -91,7 +91,7 @@ object GradientBoostedTreeClassifierExample { println("Learned classification GBT model:\n" + gbtModel.toDebugString) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala index b8cf9629bbda..e53aab7f326d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala @@ -18,24 +18,24 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.evaluation.RegressionEvaluator import org.apache.spark.ml.feature.VectorIndexer import org.apache.spark.ml.regression.{GBTRegressionModel, GBTRegressor} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object GradientBoostedTreeRegressorExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("GradientBoostedTreeRegressorExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("GradientBoostedTreeRegressorExample") + .getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Automatically identify categorical features, and index them. // Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -45,7 +45,7 @@ object GradientBoostedTreeRegressorExample { .setMaxCategories(4) .fit(data) - // Split the data into training and test sets (30% held out for testing) + // Split the data into training and test sets (30% held out for testing). val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) // Train a GBT model. @@ -54,11 +54,11 @@ object GradientBoostedTreeRegressorExample { .setFeaturesCol("indexedFeatures") .setMaxIter(10) - // Chain indexer and GBT in a Pipeline + // Chain indexer and GBT in a Pipeline. val pipeline = new Pipeline() .setStages(Array(featureIndexer, gbt)) - // Train model. This also runs the indexer. + // Train model. This also runs the indexer. val model = pipeline.fit(trainingData) // Make predictions. @@ -67,7 +67,7 @@ object GradientBoostedTreeRegressorExample { // Select example rows to display. predictions.select("prediction", "label", "features").show(5) - // Select (prediction, true label) and compute test error + // Select (prediction, true label) and compute test error. val evaluator = new RegressionEvaluator() .setLabelCol("label") .setPredictionCol("prediction") @@ -79,7 +79,7 @@ object GradientBoostedTreeRegressorExample { println("Learned regression GBT model:\n" + gbtModel.toDebugString) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala index 4cea09ba1265..950733831c3d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala @@ -18,21 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.{IndexToString, StringIndexer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object IndexToStringExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("IndexToStringExample") - val sc = new SparkContext(conf) - - val sqlContext = SQLContext.getOrCreate(sc) + val spark = SparkSession + .builder + .appName("IndexToStringExample") + .getOrCreate() // $example on$ - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, "a"), (1, "b"), (2, "c"), @@ -54,7 +53,8 @@ object IndexToStringExample { val converted = converter.transform(indexed) converted.select("id", "originalCategory").show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala index 7af011571f76..2341b36db240 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala @@ -19,15 +19,13 @@ package org.apache.spark.examples.ml // scalastyle:off println -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.clustering.KMeans -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.sql.{DataFrame, SQLContext} // $example off$ +import org.apache.spark.sql.SparkSession /** - * An example demonstrating a k-means clustering. + * An example demonstrating k-means clustering. * Run with * {{{ * bin/run-example ml.KMeansExample @@ -36,35 +34,30 @@ import org.apache.spark.sql.{DataFrame, SQLContext} object KMeansExample { def main(args: Array[String]): Unit = { - // Creates a Spark context and a SQL context - val conf = new SparkConf().setAppName(s"${this.getClass.getSimpleName}") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + // Creates a SparkSession. + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() // $example on$ - // Crates a DataFrame - val dataset: DataFrame = sqlContext.createDataFrame(Seq( - (1, Vectors.dense(0.0, 0.0, 0.0)), - (2, Vectors.dense(0.1, 0.1, 0.1)), - (3, Vectors.dense(0.2, 0.2, 0.2)), - (4, Vectors.dense(9.0, 9.0, 9.0)), - (5, Vectors.dense(9.1, 9.1, 9.1)), - (6, Vectors.dense(9.2, 9.2, 9.2)) - )).toDF("id", "features") + // Loads data. + val dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") - // Trains a k-means model - val kmeans = new KMeans() - .setK(2) - .setFeaturesCol("features") - .setPredictionCol("prediction") + // Trains a k-means model. + val kmeans = new KMeans().setK(2).setSeed(1L) val model = kmeans.fit(dataset) - // Shows the result - println("Final Centers: ") + // Evaluate clustering by computing Within Set Sum of Squared Errors. + val WSSSE = model.computeCost(dataset) + println(s"Within Set Sum of Squared Errors = $WSSSE") + + // Shows the result. + println("Cluster Centers: ") model.clusterCenters.foreach(println) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala index f9ddac77090e..22b3b0e3ad9c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala @@ -18,60 +18,51 @@ package org.apache.spark.examples.ml // scalastyle:off println -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.clustering.LDA -import org.apache.spark.mllib.linalg.{Vectors, VectorUDT} -import org.apache.spark.sql.{Row, SQLContext} -import org.apache.spark.sql.types.{StructField, StructType} // $example off$ +import org.apache.spark.sql.SparkSession /** - * An example demonstrating a LDA of ML pipeline. + * An example demonstrating LDA. * Run with * {{{ * bin/run-example ml.LDAExample * }}} */ object LDAExample { - - final val FEATURES_COL = "features" - def main(args: Array[String]): Unit = { - - val input = "data/mllib/sample_lda_data.txt" - // Creates a Spark context and a SQL context - val conf = new SparkConf().setAppName(s"${this.getClass.getSimpleName}") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + // Creates a SparkSession + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() // $example on$ - // Loads data - val rowRDD = sc.textFile(input).filter(_.nonEmpty) - .map(_.split(" ").map(_.toDouble)).map(Vectors.dense).map(Row(_)) - val schema = StructType(Array(StructField(FEATURES_COL, new VectorUDT, false))) - val dataset = sqlContext.createDataFrame(rowRDD, schema) + // Loads data. + val dataset = spark.read.format("libsvm") + .load("data/mllib/sample_lda_libsvm_data.txt") - // Trains a LDA model - val lda = new LDA() - .setK(10) - .setMaxIter(10) - .setFeaturesCol(FEATURES_COL) + // Trains a LDA model. + val lda = new LDA().setK(10).setMaxIter(10) val model = lda.fit(dataset) - val transformed = model.transform(dataset) val ll = model.logLikelihood(dataset) val lp = model.logPerplexity(dataset) + println(s"The lower bound on the log likelihood of the entire corpus: $ll") + println(s"The upper bound bound on perplexity: $lp") - // describeTopics + // Describe topics. val topics = model.describeTopics(3) - - // Shows the result + println("The topics described by their top-weighted terms:") topics.show(false) - transformed.show(false) + // Shows the result. + val transformed = model.transform(dataset) + transformed.show(false) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala index 25be87811da9..de96fb2979ad 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala @@ -22,10 +22,9 @@ import scala.language.reflectiveCalls import scopt.OptionParser -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.regression.LinearRegression -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, SparkSession} /** * An example runner for linear regression with elastic-net (mixing L1/L2) regularization. @@ -74,11 +73,11 @@ object LinearRegressionExample { s"to higher accuracy with the cost of more iterations, default: ${defaultParams.tol}") .action((x, c) => c.copy(tol = x)) opt[Double]("fracTest") - .text(s"fraction of data to hold out for testing. If given option testInput, " + + .text(s"fraction of data to hold out for testing. If given option testInput, " + s"this option is ignored. default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) opt[String]("testInput") - .text(s"input path to test dataset. If given, option fracTest is ignored." + + .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") .action((x, c) => c.copy(testInput = x)) opt[String]("dataFormat") @@ -105,13 +104,15 @@ object LinearRegressionExample { } def run(params: Params) { - val conf = new SparkConf().setAppName(s"LinearRegressionExample with $params") - val sc = new SparkContext(conf) + val spark = SparkSession + .builder + .appName(s"LinearRegressionExample with $params") + .getOrCreate() println(s"LinearRegressionExample with parameters:\n$params") // Load training and test data and cache it. - val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(sc, params.input, + val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(params.input, params.dataFormat, params.testInput, "regression", params.fracTest) val lir = new LinearRegression() @@ -136,7 +137,7 @@ object LinearRegressionExample { println("Test data results:") DecisionTreeExample.evaluateRegressionModel(lirModel, test, "label") - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala index f68aef708201..94cf2866238b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala @@ -18,22 +18,22 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.regression.LinearRegression // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object LinearRegressionWithElasticNetExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("LinearRegressionWithElasticNetExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("LinearRegressionWithElasticNetExample") + .getOrCreate() // $example on$ // Load training data - val training = sqlContext.read.format("libsvm") + val training = spark.read.format("libsvm") .load("data/mllib/sample_linear_regression_data.txt") val lr = new LinearRegression() @@ -56,7 +56,7 @@ object LinearRegressionWithElasticNetExample { println(s"r2: ${trainingSummary.r2}") // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala index a380c90662a5..c2a87e1ddfd5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala @@ -23,12 +23,11 @@ import scala.language.reflectiveCalls import scopt.OptionParser -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.{Pipeline, PipelineStage} import org.apache.spark.ml.classification.{LogisticRegression, LogisticRegressionModel} import org.apache.spark.ml.feature.StringIndexer -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, SparkSession} /** * An example runner for logistic regression with elastic-net (mixing L1/L2) regularization. @@ -81,11 +80,11 @@ object LogisticRegressionExample { s"to higher accuracy with the cost of more iterations, default: ${defaultParams.tol}") .action((x, c) => c.copy(tol = x)) opt[Double]("fracTest") - .text(s"fraction of data to hold out for testing. If given option testInput, " + + .text(s"fraction of data to hold out for testing. If given option testInput, " + s"this option is ignored. default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) opt[String]("testInput") - .text(s"input path to test dataset. If given, option fracTest is ignored." + + .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") .action((x, c) => c.copy(testInput = x)) opt[String]("dataFormat") @@ -112,16 +111,18 @@ object LogisticRegressionExample { } def run(params: Params) { - val conf = new SparkConf().setAppName(s"LogisticRegressionExample with $params") - val sc = new SparkContext(conf) + val spark = SparkSession + .builder + .appName(s"LogisticRegressionExample with $params") + .getOrCreate() println(s"LogisticRegressionExample with parameters:\n$params") // Load training and test data and cache it. - val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(sc, params.input, + val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(params.input, params.dataFormat, params.testInput, "classification", params.fracTest) - // Set up Pipeline + // Set up Pipeline. val stages = new mutable.ArrayBuffer[PipelineStage]() val labelIndexer = new StringIndexer() @@ -141,7 +142,7 @@ object LogisticRegressionExample { stages += lor val pipeline = new Pipeline().setStages(stages.toArray) - // Fit the Pipeline + // Fit the Pipeline. val startTime = System.nanoTime() val pipelineModel = pipeline.fit(training) val elapsedTime = (System.nanoTime() - startTime) / 1e9 @@ -156,7 +157,7 @@ object LogisticRegressionExample { println("Test data results:") DecisionTreeExample.evaluateClassificationModel(pipelineModel, test, "indexedLabel") - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala index 89c5edf1ace9..cd8775c94216 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala @@ -18,23 +18,23 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.classification.{BinaryLogisticRegressionSummary, LogisticRegression} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions.max object LogisticRegressionSummaryExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("LogisticRegressionSummaryExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession + .builder + .appName("LogisticRegressionSummaryExample") + .getOrCreate() + import spark.implicits._ // Load training data - val training = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val training = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val lr = new LogisticRegression() .setMaxIter(10) @@ -71,7 +71,7 @@ object LogisticRegressionSummaryExample { lrModel.setThreshold(bestThreshold) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala index 6e27571f1dc1..616263b8e9f4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala @@ -18,22 +18,22 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.classification.LogisticRegression // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object LogisticRegressionWithElasticNetExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("LogisticRegressionWithElasticNetExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("LogisticRegressionWithElasticNetExample") + .getOrCreate() // $example on$ // Load training data - val training = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val training = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val lr = new LogisticRegression() .setMaxIter(10) @@ -47,7 +47,7 @@ object LogisticRegressionWithElasticNetExample { println(s"Coefficients: ${lrModel.coefficients} Intercept: ${lrModel.intercept}") // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala index aafb5efd698e..572adce65708 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala @@ -15,23 +15,22 @@ * limitations under the License. */ -// scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.MaxAbsScaler // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object MaxAbsScalerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("MaxAbsScalerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("MaxAbsScalerExample") + .getOrCreate() // $example on$ - val dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val scaler = new MaxAbsScaler() .setInputCol("features") .setOutputCol("scaledFeatures") @@ -43,7 +42,7 @@ object MaxAbsScalerExample { val scaledData = scalerModel.transform(dataFrame) scaledData.show() // $example off$ - sc.stop() + + spark.stop() } } -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala index 9a03f69f5af0..d728019a621d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala @@ -18,20 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.MinMaxScaler // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object MinMaxScalerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("MinMaxScalerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("MinMaxScalerExample") + .getOrCreate() // $example on$ - val dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val scaler = new MinMaxScaler() .setInputCol("features") @@ -44,7 +44,8 @@ object MinMaxScalerExample { val scaledData = scalerModel.transform(dataFrame) scaledData.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala index d1441b5497a8..c1ff9ef52170 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala @@ -18,17 +18,16 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator import org.apache.spark.ml.feature.{HashingTF, Tokenizer} +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.tuning.{CrossValidator, ParamGridBuilder} -import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.Row // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * A simple example demonstrating model selection using CrossValidator. @@ -42,13 +41,14 @@ import org.apache.spark.sql.SQLContext object ModelSelectionViaCrossValidationExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("ModelSelectionViaCrossValidationExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("ModelSelectionViaCrossValidationExample") + .getOrCreate() // $example on$ // Prepare training data from a list of (id, text, label) tuples. - val training = sqlContext.createDataFrame(Seq( + val training = spark.createDataFrame(Seq( (0L, "a b c d e spark", 1.0), (1L, "b d", 0.0), (2L, "spark f g h", 1.0), @@ -98,7 +98,7 @@ object ModelSelectionViaCrossValidationExample { val cvModel = cv.fit(training) // Prepare test documents, which are unlabeled (id, text) tuples. - val test = sqlContext.createDataFrame(Seq( + val test = spark.createDataFrame(Seq( (4L, "spark i j k"), (5L, "l m n"), (6L, "mapreduce spark"), @@ -114,7 +114,7 @@ object ModelSelectionViaCrossValidationExample { } // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala index fcad17a81758..75fef2922adb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala @@ -17,13 +17,12 @@ package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.evaluation.RegressionEvaluator import org.apache.spark.ml.regression.LinearRegression import org.apache.spark.ml.tuning.{ParamGridBuilder, TrainValidationSplit} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * A simple example demonstrating model selection using TrainValidationSplit. @@ -36,13 +35,14 @@ import org.apache.spark.sql.SQLContext object ModelSelectionViaTrainValidationSplitExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("ModelSelectionViaTrainValidationSplitExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("ModelSelectionViaTrainValidationSplitExample") + .getOrCreate() // $example on$ // Prepare training and test data. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") val Array(training, test) = data.randomSplit(Array(0.9, 0.1), seed = 12345) val lr = new LinearRegression() @@ -75,6 +75,6 @@ object ModelSelectionViaTrainValidationSplitExample { .show() // $example off$ - sc.stop() + spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala index d7d1e82f6f84..0e780fb7d342 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala @@ -18,12 +18,11 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.classification.MultilayerPerceptronClassifier import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * An example for Multilayer Perceptron Classification. @@ -31,13 +30,14 @@ import org.apache.spark.sql.SQLContext object MultilayerPerceptronClassifierExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("MultilayerPerceptronClassifierExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("MultilayerPerceptronClassifierExample") + .getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - val data = sqlContext.read.format("libsvm") + val data = spark.read.format("libsvm") .load("data/mllib/sample_multiclass_classification_data.txt") // Split the data into train and test val splits = data.randomSplit(Array(0.6, 0.4), seed = 1234L) @@ -63,7 +63,7 @@ object MultilayerPerceptronClassifierExample { println("Precision:" + evaluator.evaluate(predictionAndLabels)) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala index 77b913aaa3fa..e0b52e7a367f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala @@ -18,20 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.NGram // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object NGramExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("NGramExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("NGramExample") + .getOrCreate() // $example on$ - val wordDataFrame = sqlContext.createDataFrame(Seq( + val wordDataFrame = spark.createDataFrame(Seq( (0, Array("Hi", "I", "heard", "about", "Spark")), (1, Array("I", "wish", "Java", "could", "use", "case", "classes")), (2, Array("Logistic", "regression", "models", "are", "neat")) @@ -41,7 +41,8 @@ object NGramExample { val ngramDataFrame = ngram.transform(wordDataFrame) ngramDataFrame.take(3).map(_.getAs[Stream[String]]("ngrams").toList).foreach(println) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala index 5ea1270c9781..90cdebfcb08d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala @@ -18,21 +18,21 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ -import org.apache.spark.ml.classification.{NaiveBayes} +import org.apache.spark.ml.classification.NaiveBayes import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object NaiveBayesExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("NaiveBayesExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("NaiveBayesExample") + .getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Split the data into training and test sets (30% held out for testing) val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) @@ -53,6 +53,8 @@ object NaiveBayesExample { val precision = evaluator.evaluate(predictions) println("Precision:" + precision) // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala index 6b33c16c7403..75ba33a7e7fc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala @@ -18,20 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.Normalizer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object NormalizerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("NormalizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("NormalizerExample") + .getOrCreate() // $example on$ - val dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Normalize each Vector using $L^1$ norm. val normalizer = new Normalizer() @@ -46,7 +46,8 @@ object NormalizerExample { val lInfNormData = normalizer.transform(dataFrame, normalizer.p -> Double.PositiveInfinity) lInfNormData.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala index cb9fe65a85e8..4aa649b1332c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala @@ -18,20 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.{OneHotEncoder, StringIndexer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object OneHotEncoderExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("OneHotEncoderExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("OneHotEncoderExample") + .getOrCreate() // $example on$ - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, "a"), (1, "b"), (2, "c"), @@ -52,7 +52,8 @@ object OneHotEncoderExample { val encoded = encoder.transform(indexed) encoded.select("id", "categoryVec").show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala index 0b5d31c0ff90..0b333cf62941 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala @@ -18,171 +18,63 @@ // scalastyle:off println package org.apache.spark.examples.ml -import java.util.concurrent.TimeUnit.{NANOSECONDS => NANO} - -import scopt.OptionParser - -import org.apache.spark.{SparkConf, SparkContext} // $example on$ -import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.classification.{LogisticRegression, OneVsRest} -import org.apache.spark.ml.util.MetadataUtils -import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator import org.apache.spark.sql.DataFrame // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** - * An example runner for Multiclass to Binary Reduction with One Vs Rest. - * The example uses Logistic Regression as the base classifier. All parameters that - * can be specified on the base classifier can be passed in to the runner options. + * An example of Multiclass to Binary Reduction with One Vs Rest, + * using Logistic Regression as the base classifier. * Run with * {{{ - * ./bin/run-example ml.OneVsRestExample [options] - * }}} - * For local mode, run - * {{{ - * ./bin/spark-submit --class org.apache.spark.examples.ml.OneVsRestExample --driver-memory 1g - * [examples JAR path] [options] + * ./bin/run-example ml.OneVsRestExample * }}} - * If you use it as a template to create your own app, please use `spark-submit` to submit your app. */ -object OneVsRestExample { - - case class Params private[ml] ( - input: String = null, - testInput: Option[String] = None, - maxIter: Int = 100, - tol: Double = 1E-6, - fitIntercept: Boolean = true, - regParam: Option[Double] = None, - elasticNetParam: Option[Double] = None, - fracTest: Double = 0.2) extends AbstractParams[Params] +object OneVsRestExample { def main(args: Array[String]) { - val defaultParams = Params() - - val parser = new OptionParser[Params]("OneVsRest Example") { - head("OneVsRest Example: multiclass to binary reduction using OneVsRest") - opt[String]("input") - .text("input path to labeled examples. This path must be specified") - .required() - .action((x, c) => c.copy(input = x)) - opt[Double]("fracTest") - .text(s"fraction of data to hold out for testing. If given option testInput, " + - s"this option is ignored. default: ${defaultParams.fracTest}") - .action((x, c) => c.copy(fracTest = x)) - opt[String]("testInput") - .text("input path to test dataset. If given, option fracTest is ignored") - .action((x, c) => c.copy(testInput = Some(x))) - opt[Int]("maxIter") - .text(s"maximum number of iterations for Logistic Regression." + - s" default: ${defaultParams.maxIter}") - .action((x, c) => c.copy(maxIter = x)) - opt[Double]("tol") - .text(s"the convergence tolerance of iterations for Logistic Regression." + - s" default: ${defaultParams.tol}") - .action((x, c) => c.copy(tol = x)) - opt[Boolean]("fitIntercept") - .text(s"fit intercept for Logistic Regression." + - s" default: ${defaultParams.fitIntercept}") - .action((x, c) => c.copy(fitIntercept = x)) - opt[Double]("regParam") - .text(s"the regularization parameter for Logistic Regression.") - .action((x, c) => c.copy(regParam = Some(x))) - opt[Double]("elasticNetParam") - .text(s"the ElasticNet mixing parameter for Logistic Regression.") - .action((x, c) => c.copy(elasticNetParam = Some(x))) - checkConfig { params => - if (params.fracTest < 0 || params.fracTest >= 1) { - failure(s"fracTest ${params.fracTest} value incorrect; should be in [0,1).") - } else { - success - } - } - } - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - sys.exit(1) - } - } - - private def run(params: Params) { - val conf = new SparkConf().setAppName(s"OneVsRestExample with $params") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName(s"OneVsRestExample") + .getOrCreate() // $example on$ - val inputData = sqlContext.read.format("libsvm").load(params.input) - // compute the train/test split: if testInput is not provided use part of input. - val data = params.testInput match { - case Some(t) => - // compute the number of features in the training set. - val numFeatures = inputData.first().getAs[Vector](1).size - val testData = sqlContext.read.option("numFeatures", numFeatures.toString) - .format("libsvm").load(t) - Array[DataFrame](inputData, testData) - case None => - val f = params.fracTest - inputData.randomSplit(Array(1 - f, f), seed = 12345) - } - val Array(train, test) = data.map(_.cache()) + // load data file. + val inputData: DataFrame = spark.read.format("libsvm") + .load("data/mllib/sample_multiclass_classification_data.txt") + + // generate the train/test split. + val Array(train, test) = inputData.randomSplit(Array(0.8, 0.2)) // instantiate the base classifier val classifier = new LogisticRegression() - .setMaxIter(params.maxIter) - .setTol(params.tol) - .setFitIntercept(params.fitIntercept) - - // Set regParam, elasticNetParam if specified in params - params.regParam.foreach(classifier.setRegParam) - params.elasticNetParam.foreach(classifier.setElasticNetParam) + .setMaxIter(10) + .setTol(1E-6) + .setFitIntercept(true) // instantiate the One Vs Rest Classifier. - - val ovr = new OneVsRest() - ovr.setClassifier(classifier) + val ovr = new OneVsRest().setClassifier(classifier) // train the multiclass model. - val (trainingDuration, ovrModel) = time(ovr.fit(train)) + val ovrModel = ovr.fit(train) // score the model on test data. - val (predictionDuration, predictions) = time(ovrModel.transform(test)) - - // evaluate the model - val predictionsAndLabels = predictions.select("prediction", "label") - .rdd.map(row => (row.getDouble(0), row.getDouble(1))) - - val metrics = new MulticlassMetrics(predictionsAndLabels) - - val confusionMatrix = metrics.confusionMatrix + val predictions = ovrModel.transform(test) - // compute the false positive rate per label - val predictionColSchema = predictions.schema("prediction") - val numClasses = MetadataUtils.getNumClasses(predictionColSchema).get - val fprs = Range(0, numClasses).map(p => (p, metrics.falsePositiveRate(p.toDouble))) + // obtain evaluator. + val evaluator = new MulticlassClassificationEvaluator() + .setMetricName("precision") - println(s" Training Time ${trainingDuration} sec\n") - - println(s" Prediction Time ${predictionDuration} sec\n") - - println(s" Confusion Matrix\n ${confusionMatrix.toString}\n") - - println("label\tfpr") - - println(fprs.map {case (label, fpr) => label + "\t" + fpr}.mkString("\n")) + // compute the classification error on test data. + val precision = evaluator.evaluate(predictions) + println(s"Test Error : ${1 - precision}") // $example off$ - sc.stop() + spark.stop() } - private def time[R](block: => R): (Long, R) = { - val t0 = System.nanoTime() - val result = block // call-by-name - val t1 = System.nanoTime() - (NANO.toSeconds(t1 - t0), result) - } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala index 535652ec6c79..dca96eea2ba4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala @@ -18,18 +18,18 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.PCA -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object PCAExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("PCAExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("PCAExample") + .getOrCreate() // $example on$ val data = Array( @@ -37,7 +37,7 @@ object PCAExample { Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0), Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0) ) - val df = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") + val df = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") val pca = new PCA() .setInputCol("features") .setOutputCol("pcaFeatures") @@ -47,7 +47,8 @@ object PCAExample { val result = pcaDF.select("pcaFeatures") result.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala index 6c29063626ba..b16692b1fa36 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala @@ -18,26 +18,26 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.{Pipeline, PipelineModel} import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.feature.{HashingTF, Tokenizer} -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector import org.apache.spark.sql.Row // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object PipelineExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("PipelineExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("PipelineExample") + .getOrCreate() // $example on$ // Prepare training documents from a list of (id, text, label) tuples. - val training = sqlContext.createDataFrame(Seq( + val training = spark.createDataFrame(Seq( (0L, "a b c d e spark", 1.0), (1L, "b d", 0.0), (2L, "spark f g h", 1.0), @@ -71,7 +71,7 @@ object PipelineExample { val sameModel = PipelineModel.load("/tmp/spark-logistic-regression-model") // Prepare test documents, which are unlabeled (id, text) tuples. - val test = sqlContext.createDataFrame(Seq( + val test = spark.createDataFrame(Seq( (4L, "spark i j k"), (5L, "l m n"), (6L, "mapreduce spark"), @@ -87,7 +87,7 @@ object PipelineExample { } // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala index 3014008ea0ce..54d2e6b36d14 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala @@ -18,18 +18,18 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.PolynomialExpansion -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object PolynomialExpansionExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("PolynomialExpansionExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("PolynomialExpansionExample") + .getOrCreate() // $example on$ val data = Array( @@ -37,7 +37,7 @@ object PolynomialExpansionExample { Vectors.dense(0.0, 0.0), Vectors.dense(0.6, -1.1) ) - val df = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") + val df = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") val polynomialExpansion = new PolynomialExpansion() .setInputCol("features") .setOutputCol("polyFeatures") @@ -45,7 +45,8 @@ object PolynomialExpansionExample { val polyDF = polynomialExpansion.transform(df) polyDF.select("polyFeatures").take(3).foreach(println) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala index e64e673a485e..1a1651559416 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala @@ -15,25 +15,24 @@ * limitations under the License. */ -// scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.QuantileDiscretizer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object QuantileDiscretizerExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("QuantileDiscretizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession + .builder + .appName("QuantileDiscretizerExample") + .getOrCreate() + import spark.implicits._ // $example on$ val data = Array((0, 18.0), (1, 19.0), (2, 8.0), (3, 5.0), (4, 2.2)) - val df = sc.parallelize(data).toDF("id", "hour") + val df = spark.createDataFrame(data).toDF("id", "hour") val discretizer = new QuantileDiscretizer() .setInputCol("hour") @@ -43,7 +42,7 @@ object QuantileDiscretizerExample { val result = discretizer.fit(df).transform(df) result.show() // $example off$ - sc.stop() + + spark.stop() } } -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala index bec831d51c58..9ea492014644 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala @@ -18,20 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.RFormula // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object RFormulaExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("RFormulaExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("RFormulaExample") + .getOrCreate() // $example on$ - val dataset = sqlContext.createDataFrame(Seq( + val dataset = spark.createDataFrame(Seq( (7, "US", 18, 1.0), (8, "CA", 12, 0.0), (9, "NZ", 15, 0.0) @@ -43,7 +43,8 @@ object RFormulaExample { val output = formula.fit(dataset).transform(dataset) output.select("features", "label").show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala index 6c9b52cf259e..cccc4a6ea26b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala @@ -18,24 +18,24 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.{RandomForestClassificationModel, RandomForestClassifier} import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator import org.apache.spark.ml.feature.{IndexToString, StringIndexer, VectorIndexer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object RandomForestClassifierExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("RandomForestClassifierExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("RandomForestClassifierExample") + .getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Index labels, adding metadata to the label column. // Fit on whole dataset to include all labels in index. @@ -51,7 +51,7 @@ object RandomForestClassifierExample { .setMaxCategories(4) .fit(data) - // Split the data into training and test sets (30% held out for testing) + // Split the data into training and test sets (30% held out for testing). val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) // Train a RandomForest model. @@ -66,11 +66,11 @@ object RandomForestClassifierExample { .setOutputCol("predictedLabel") .setLabels(labelIndexer.labels) - // Chain indexers and forest in a Pipeline + // Chain indexers and forest in a Pipeline. val pipeline = new Pipeline() .setStages(Array(labelIndexer, featureIndexer, rf, labelConverter)) - // Train model. This also runs the indexers. + // Train model. This also runs the indexers. val model = pipeline.fit(trainingData) // Make predictions. @@ -79,7 +79,7 @@ object RandomForestClassifierExample { // Select example rows to display. predictions.select("predictedLabel", "label", "features").show(5) - // Select (prediction, true label) and compute test error + // Select (prediction, true label) and compute test error. val evaluator = new MulticlassClassificationEvaluator() .setLabelCol("indexedLabel") .setPredictionCol("prediction") @@ -91,7 +91,7 @@ object RandomForestClassifierExample { println("Learned classification forest model:\n" + rfModel.toDebugString) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala index 7a00d99dfe53..2419dc49cd51 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala @@ -23,13 +23,12 @@ import scala.language.reflectiveCalls import scopt.OptionParser -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.{Pipeline, PipelineStage} import org.apache.spark.ml.classification.{RandomForestClassificationModel, RandomForestClassifier} import org.apache.spark.ml.feature.{StringIndexer, VectorIndexer} import org.apache.spark.ml.regression.{RandomForestRegressionModel, RandomForestRegressor} -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, SparkSession} /** @@ -37,7 +36,7 @@ import org.apache.spark.sql.DataFrame * {{{ * ./bin/run-example ml.RandomForestExample [options] * }}} - * Decision Trees and ensembles can take a large amount of memory. If the run-example command + * Decision Trees and ensembles can take a large amount of memory. If the run-example command * above fails, try running via spark-submit and specifying the amount of memory as at least 1g. * For local mode, run * {{{ @@ -94,7 +93,7 @@ object RandomForestExample { s" default: ${defaultParams.numTrees}") .action((x, c) => c.copy(featureSubsetStrategy = x)) opt[Double]("fracTest") - .text(s"fraction of data to hold out for testing. If given option testInput, " + + .text(s"fraction of data to hold out for testing. If given option testInput, " + s"this option is ignored. default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) opt[Boolean]("cacheNodeIds") @@ -115,7 +114,7 @@ object RandomForestExample { s"default: ${defaultParams.checkpointInterval}") .action((x, c) => c.copy(checkpointInterval = x)) opt[String]("testInput") - .text(s"input path to test dataset. If given, option fracTest is ignored." + + .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") .action((x, c) => c.copy(testInput = x)) opt[String]("dataFormat") @@ -142,18 +141,21 @@ object RandomForestExample { } def run(params: Params) { - val conf = new SparkConf().setAppName(s"RandomForestExample with $params") - val sc = new SparkContext(conf) - params.checkpointDir.foreach(sc.setCheckpointDir) + val spark = SparkSession + .builder + .appName(s"RandomForestExample with $params") + .getOrCreate() + + params.checkpointDir.foreach(spark.sparkContext.setCheckpointDir) val algo = params.algo.toLowerCase println(s"RandomForestExample with parameters:\n$params") // Load training and test data and cache it. - val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(sc, params.input, + val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(params.input, params.dataFormat, params.testInput, algo, params.fracTest) - // Set up Pipeline + // Set up Pipeline. val stages = new mutable.ArrayBuffer[PipelineStage]() // (1) For classification, re-index classes. val labelColName = if (algo == "classification") "indexedLabel" else "label" @@ -170,7 +172,7 @@ object RandomForestExample { .setOutputCol("indexedFeatures") .setMaxCategories(10) stages += featuresIndexer - // (3) Learn Random Forest + // (3) Learn Random Forest. val dt = algo match { case "classification" => new RandomForestClassifier() @@ -201,13 +203,13 @@ object RandomForestExample { stages += dt val pipeline = new Pipeline().setStages(stages.toArray) - // Fit the Pipeline + // Fit the Pipeline. val startTime = System.nanoTime() val pipelineModel = pipeline.fit(training) val elapsedTime = (System.nanoTime() - startTime) / 1e9 println(s"Training time: $elapsedTime seconds") - // Get the trained Random Forest from the fitted PipelineModel + // Get the trained Random Forest from the fitted PipelineModel. algo match { case "classification" => val rfModel = pipelineModel.stages.last.asInstanceOf[RandomForestClassificationModel] @@ -226,7 +228,7 @@ object RandomForestExample { case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") } - // Evaluate model on training, test data + // Evaluate model on training, test data. algo match { case "classification" => println("Training data results:") @@ -242,7 +244,7 @@ object RandomForestExample { throw new IllegalArgumentException("Algo ${params.algo} not supported.") } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala index 4d2db017f346..9a0a001c26ef 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala @@ -18,24 +18,24 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.evaluation.RegressionEvaluator import org.apache.spark.ml.feature.VectorIndexer import org.apache.spark.ml.regression.{RandomForestRegressionModel, RandomForestRegressor} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object RandomForestRegressorExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("RandomForestRegressorExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("RandomForestRegressorExample") + .getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Automatically identify categorical features, and index them. // Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -45,7 +45,7 @@ object RandomForestRegressorExample { .setMaxCategories(4) .fit(data) - // Split the data into training and test sets (30% held out for testing) + // Split the data into training and test sets (30% held out for testing). val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) // Train a RandomForest model. @@ -53,11 +53,11 @@ object RandomForestRegressorExample { .setLabelCol("label") .setFeaturesCol("indexedFeatures") - // Chain indexer and forest in a Pipeline + // Chain indexer and forest in a Pipeline. val pipeline = new Pipeline() .setStages(Array(featureIndexer, rf)) - // Train model. This also runs the indexer. + // Train model. This also runs the indexer. val model = pipeline.fit(trainingData) // Make predictions. @@ -66,7 +66,7 @@ object RandomForestRegressorExample { // Select example rows to display. predictions.select("prediction", "label", "features").show(5) - // Select (prediction, true label) and compute test error + // Select (prediction, true label) and compute test error. val evaluator = new RegressionEvaluator() .setLabelCol("label") .setPredictionCol("prediction") @@ -78,7 +78,7 @@ object RandomForestRegressorExample { println("Learned regression forest model:\n" + rfModel.toDebugString) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala index 202925acadff..bb4587b82cb3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala @@ -18,20 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.SQLTransformer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object SQLTransformerExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("SQLTransformerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("SQLTransformerExample") + .getOrCreate() // $example on$ - val df = sqlContext.createDataFrame( + val df = spark.createDataFrame( Seq((0, 1.0, 3.0), (2, 2.0, 5.0))).toDF("id", "v1", "v2") val sqlTrans = new SQLTransformer().setStatement( @@ -39,6 +39,8 @@ object SQLTransformerExample { sqlTrans.transform(df).show() // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index f4d1fe57856a..29f1f509608a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -18,12 +18,11 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} /** * A simple example demonstrating ways to specify parameters for Estimators and Transformers. @@ -35,21 +34,22 @@ import org.apache.spark.sql.{Row, SQLContext} object SimpleParamsExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("SimpleParamsExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession + .builder + .appName("SimpleParamsExample") + .getOrCreate() + import spark.implicits._ // Prepare training data. - // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes + // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes // into DataFrames, where it uses the case class metadata to infer the schema. - val training = sc.parallelize(Seq( + val training = spark.createDataFrame(Seq( LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)))) - // Create a LogisticRegression instance. This instance is an Estimator. + // Create a LogisticRegression instance. This instance is an Estimator. val lr = new LogisticRegression() // Print out the parameters, documentation, and any default values. println("LogisticRegression parameters:\n" + lr.explainParams() + "\n") @@ -58,8 +58,8 @@ object SimpleParamsExample { lr.setMaxIter(10) .setRegParam(0.01) - // Learn a LogisticRegression model. This uses the parameters stored in lr. - val model1 = lr.fit(training.toDF()) + // Learn a LogisticRegression model. This uses the parameters stored in lr. + val model1 = lr.fit(training) // Since model1 is a Model (i.e., a Transformer produced by an Estimator), // we can view the parameters it used during fit(). // This prints the parameter (name: value) pairs, where names are unique IDs for this @@ -69,8 +69,8 @@ object SimpleParamsExample { // We may alternatively specify parameters using a ParamMap, // which supports several methods for specifying parameters. val paramMap = ParamMap(lr.maxIter -> 20) - paramMap.put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. - paramMap.put(lr.regParam -> 0.1, lr.thresholds -> Array(0.45, 0.55)) // Specify multiple Params. + paramMap.put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. + paramMap.put(lr.regParam -> 0.1, lr.thresholds -> Array(0.5, 0.5)) // Specify multiple Params. // One can also combine ParamMaps. val paramMap2 = ParamMap(lr.probabilityCol -> "myProbability") // Change output column name @@ -82,7 +82,7 @@ object SimpleParamsExample { println("Model 2 was fit using parameters: " + model2.parent.extractParamMap()) // Prepare test data. - val test = sc.parallelize(Seq( + val test = spark.createDataFrame(Seq( LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) @@ -91,14 +91,14 @@ object SimpleParamsExample { // LogisticRegressionModel.transform will only use the 'features' column. // Note that model2.transform() outputs a 'myProbability' column instead of the usual // 'probability' column since we renamed the lr.probabilityCol parameter previously. - model2.transform(test.toDF()) + model2.transform(test) .select("features", "label", "myProbability", "prediction") .collect() .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) => println(s"($features, $label) -> prob=$prob, prediction=$prediction") } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index 960280137cbf..0b2a058bb61a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -20,12 +20,11 @@ package org.apache.spark.examples.ml import scala.beans.BeanInfo -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.feature.{HashingTF, Tokenizer} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.sql.{Row, SparkSession} @BeanInfo case class LabeledDocument(id: Long, text: String, label: Double) @@ -43,13 +42,14 @@ case class Document(id: Long, text: String) object SimpleTextClassificationPipeline { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("SimpleTextClassificationPipeline") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession + .builder + .appName("SimpleTextClassificationPipeline") + .getOrCreate() + import spark.implicits._ // Prepare training documents, which are labeled. - val training = sc.parallelize(Seq( + val training = spark.createDataFrame(Seq( LabeledDocument(0L, "a b c d e spark", 1.0), LabeledDocument(1L, "b d", 0.0), LabeledDocument(2L, "spark f g h", 1.0), @@ -73,7 +73,7 @@ object SimpleTextClassificationPipeline { val model = pipeline.fit(training.toDF()) // Prepare test documents, which are unlabeled. - val test = sc.parallelize(Seq( + val test = spark.createDataFrame(Seq( Document(4L, "spark i j k"), Document(5L, "l m n"), Document(6L, "spark hadoop spark"), @@ -87,7 +87,7 @@ object SimpleTextClassificationPipeline { println(s"($id, $text) --> prob=$prob, prediction=$prediction") } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala index e3439677e78d..4d668e8ab967 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala @@ -18,20 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.StandardScaler // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object StandardScalerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("StandardScalerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("StandardScalerExample") + .getOrCreate() // $example on$ - val dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val scaler = new StandardScaler() .setInputCol("features") @@ -46,7 +46,8 @@ object StandardScalerExample { val scaledData = scalerModel.transform(dataFrame) scaledData.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala index 8199be12c155..fb1a43e962cd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala @@ -18,31 +18,32 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.StopWordsRemover // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object StopWordsRemoverExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("StopWordsRemoverExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("StopWordsRemoverExample") + .getOrCreate() // $example on$ val remover = new StopWordsRemover() .setInputCol("raw") .setOutputCol("filtered") - val dataSet = sqlContext.createDataFrame(Seq( + val dataSet = spark.createDataFrame(Seq( (0, Seq("I", "saw", "the", "red", "baloon")), (1, Seq("Mary", "had", "a", "little", "lamb")) )).toDF("id", "raw") remover.transform(dataSet).show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala index 3f0e870c8dc6..63f273e87a20 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala @@ -18,20 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.StringIndexer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object StringIndexerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("StringIndexerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("StringIndexerExample") + .getOrCreate() // $example on$ - val df = sqlContext.createDataFrame( + val df = spark.createDataFrame( Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")) ).toDF("id", "category") @@ -42,7 +42,8 @@ object StringIndexerExample { val indexed = indexer.fit(df).transform(df) indexed.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala index 396f073e6b32..33b5daec5978 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala @@ -18,21 +18,21 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.{HashingTF, IDF, Tokenizer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object TfIdfExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("TfIdfExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("TfIdfExample") + .getOrCreate() // $example on$ - val sentenceData = sqlContext.createDataFrame(Seq( + val sentenceData = spark.createDataFrame(Seq( (0, "Hi I heard about Spark"), (0, "I wish Java could use case classes"), (1, "Logistic regression models are neat") @@ -50,6 +50,8 @@ object TfIdfExample { val rescaledData = idfModel.transform(featurizedData) rescaledData.select("features", "label").take(3).foreach(println) // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala index c667728d6326..1c70dc700b91 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala @@ -18,20 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.{RegexTokenizer, Tokenizer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object TokenizerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("TokenizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("TokenizerExample") + .getOrCreate() // $example on$ - val sentenceDataFrame = sqlContext.createDataFrame(Seq( + val sentenceDataFrame = spark.createDataFrame(Seq( (0, "Hi I heard about Spark"), (1, "I wish Java could use case classes"), (2, "Logistic,regression,models,are,neat") @@ -48,7 +48,8 @@ object TokenizerExample { val regexTokenized = regexTokenizer.transform(sentenceDataFrame) regexTokenized.select("words", "label").take(3).foreach(println) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala index 768a8c069047..8910470c1cf7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala @@ -18,21 +18,21 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.VectorAssembler -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object VectorAssemblerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("VectorAssemblerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("VectorAssemblerExample") + .getOrCreate() // $example on$ - val dataset = sqlContext.createDataFrame( + val dataset = spark.createDataFrame( Seq((0, 18, 1.0, Vectors.dense(0.0, 10.0, 0.5), 1.0)) ).toDF("id", "hour", "mobile", "userFeatures", "clicked") @@ -43,7 +43,8 @@ object VectorAssemblerExample { val output = assembler.transform(dataset) println(output.select("features", "clicked").first()) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala index 3bef37ba360b..afa761aee0b9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala @@ -18,20 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.VectorIndexer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object VectorIndexerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("VectorIndexerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("VectorIndexerExample") + .getOrCreate() // $example on$ - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val indexer = new VectorIndexer() .setInputCol("features") @@ -48,7 +48,8 @@ object VectorIndexerExample { val indexedData = indexerModel.transform(data) indexedData.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala index 01377d80e7e5..85dd5c27766c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala @@ -18,31 +18,32 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ +import java.util.Arrays + import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute} import org.apache.spark.ml.feature.VectorSlicer -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.sql.Row import org.apache.spark.sql.types.StructType // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object VectorSlicerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("VectorSlicerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("VectorSlicerExample") + .getOrCreate() // $example on$ - val data = Array(Row(Vectors.dense(-2.0, 2.3, 0.0))) + val data = Arrays.asList(Row(Vectors.dense(-2.0, 2.3, 0.0))) val defaultAttr = NumericAttribute.defaultAttr val attrs = Array("f1", "f2", "f3").map(defaultAttr.withName) val attrGroup = new AttributeGroup("userFeatures", attrs.asInstanceOf[Array[Attribute]]) - val dataRDD = sc.parallelize(data) - val dataset = sqlContext.createDataFrame(dataRDD, StructType(Array(attrGroup.toStructField()))) + val dataset = spark.createDataFrame(data, StructType(Array(attrGroup.toStructField()))) val slicer = new VectorSlicer().setInputCol("userFeatures").setOutputCol("features") @@ -52,7 +53,8 @@ object VectorSlicerExample { val output = slicer.transform(dataset) println(output.select("userFeatures", "features").first()) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala index e77aa59ba32b..9ac562360729 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala @@ -18,21 +18,21 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.Word2Vec // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object Word2VecExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("Word2Vec example") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("Word2Vec example") + .getOrCreate() // $example on$ // Input data: Each row is a bag of words from a sentence or document. - val documentDF = sqlContext.createDataFrame(Seq( + val documentDF = spark.createDataFrame(Seq( "Hi I heard about Spark".split(" "), "I wish Java could use case classes".split(" "), "Logistic regression models are neat".split(" ") @@ -48,6 +48,8 @@ object Word2VecExample { val result = model.transform(documentDF) result.select("result").take(3).foreach(println) // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index e89d555884dd..7651aade493a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -27,7 +27,7 @@ import org.apache.spark.ml.feature.{CountVectorizer, CountVectorizerModel, Regex import org.apache.spark.mllib.clustering.{DistributedLDAModel, EMLDAOptimizer, LDA, OnlineLDAOptimizer} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} /** * An example Latent Dirichlet Allocation (LDA) app. Run with @@ -189,8 +189,10 @@ object LDAExample { vocabSize: Int, stopwordFile: String): (RDD[(Long, Vector)], Array[String], Long) = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession + .builder + .getOrCreate() + import spark.implicits._ // Get dataset of document texts // One document per line in each text file. If the input consists of many small files, diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala index fdb01b86dd78..781a934df663 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala @@ -18,22 +18,22 @@ // scalastyle:off println package org.apache.spark.examples.mllib -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.mllib.evaluation.{RankingMetrics, RegressionMetrics} import org.apache.spark.mllib.recommendation.{ALS, Rating} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object RankingMetricsExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("RankingMetricsExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession + .builder + .appName("RankingMetricsExample") + .getOrCreate() + import spark.implicits._ // $example on$ // Read in the ratings data - val ratings = sc.textFile("data/mllib/sample_movielens_data.txt").map { line => + val ratings = spark.read.text("data/mllib/sample_movielens_data.txt").rdd.map { line => val fields = line.split("::") Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble - 2.5) }.cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala index add634c957b4..abeaaa00b5a4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala @@ -18,22 +18,25 @@ package org.apache.spark.examples.mllib -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.mllib.evaluation.RegressionMetrics -import org.apache.spark.mllib.regression.LinearRegressionWithSGD -import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.{LabeledPoint, LinearRegressionWithSGD} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object RegressionMetricsExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("RegressionMetricsExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("RegressionMetricsExample") + .getOrCreate() // $example on$ // Load the data - val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_linear_regression_data.txt").cache() + val data = spark + .read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") + .rdd.map(row => LabeledPoint(row.getDouble(0), row.get(1).asInstanceOf[Vector])) + .cache() // Build the model val numIterations = 100 @@ -61,6 +64,8 @@ object RegressionMetricsExample { // Explained variance println(s"Explained variance = ${metrics.explainedVariance}") // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index 8ce4427c5399..1b019fbb5177 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -18,7 +18,6 @@ // scalastyle:off println package org.apache.spark.examples.sql -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql.{SaveMode, SparkSession} // One method for defining the schema of an RDD is to make a case class with the desired column @@ -27,17 +26,18 @@ case class Record(key: Int, value: String) object RDDRelation { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("RDDRelation") - val sc = new SparkContext(sparkConf) - val spark = new SparkSession(sc) + val spark = SparkSession + .builder + .appName("RDDRelation") + .getOrCreate() // Importing the SparkSession gives access to all the SQL functions and implicit conversions. import spark.implicits._ - val df = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))).toDF() - // Any RDD containing case classes can be registered as a table. The schema of the table is - // automatically inferred using scala reflection. - df.registerTempTable("records") + val df = spark.createDataFrame((1 to 100).map(i => Record(i, s"val_$i"))) + // Any RDD containing case classes can be used to create a temporary view. The schema of the + // view is automatically inferred using scala reflection. + df.createOrReplaceTempView("records") // Once tables have been registered, you can run SQL queries over them. println("Result of SELECT *:") @@ -66,11 +66,11 @@ object RDDRelation { // Queries can be run using the DSL on parquet files just like the original RDD. parquetFile.where($"key" === 1).select($"value".as("a")).collect().foreach(println) - // These files can also be registered as tables. - parquetFile.registerTempTable("parquetFile") + // These files can also be used to create a temporary view. + parquetFile.createOrReplaceTempView("parquetFile") spark.sql("SELECT * FROM parquetFile").collect().foreach(println) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index ff33091621c1..7293cb51b261 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -36,15 +36,19 @@ object HiveFromSpark { def main(args: Array[String]) { val sparkConf = new SparkConf().setAppName("HiveFromSpark") - val sc = new SparkContext(sparkConf) // A hive context adds support for finding tables in the MetaStore and writing queries // using HiveQL. Users who do not have an existing Hive deployment can still create a // HiveContext. When not configured by the hive-site.xml, the context automatically // creates metastore_db and warehouse in the current directory. - val sparkSession = SparkSession.withHiveSupport(sc) - import sparkSession.implicits._ - import sparkSession.sql + val spark = SparkSession.builder + .config(sparkConf) + .enableHiveSupport() + .getOrCreate() + val sc = spark.sparkContext + + import spark.implicits._ + import spark.sql sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") sql(s"LOAD DATA LOCAL INPATH '${kv1File.getAbsolutePath}' INTO TABLE src") @@ -66,15 +70,15 @@ object HiveFromSpark { case Row(key: Int, value: String) => s"Key: $key, Value: $value" } - // You can also register RDDs as temporary tables within a HiveContext. + // You can also use RDDs to create temporary views within a HiveContext. val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) - rdd.toDF().registerTempTable("records") + rdd.toDF().createOrReplaceTempView("records") // Queries can then join RDD data with data stored in Hive. println("Result of SELECT *:") sql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index 1bcd85e1d533..acbcb0c4b7b7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -23,11 +23,12 @@ import java.nio.charset.Charset import com.google.common.io.Files -import org.apache.spark.{Accumulator, SparkConf, SparkContext} +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Seconds, StreamingContext, Time} import org.apache.spark.util.IntParam +import org.apache.spark.util.LongAccumulator /** * Use this singleton to get or register a Broadcast variable. @@ -54,13 +55,13 @@ object WordBlacklist { */ object DroppedWordsCounter { - @volatile private var instance: Accumulator[Long] = null + @volatile private var instance: LongAccumulator = null - def getInstance(sc: SparkContext): Accumulator[Long] = { + def getInstance(sc: SparkContext): LongAccumulator = { if (instance == null) { synchronized { if (instance == null) { - instance = sc.accumulator(0L, "WordsInBlacklistCounter") + instance = sc.longAccumulator("WordsInBlacklistCounter") } } } @@ -124,7 +125,7 @@ object RecoverableNetworkWordCount { // Use blacklist to drop words and use droppedWordsCounter to count them val counts = rdd.filter { case (word, count) => if (blacklist.value.contains(word)) { - droppedWordsCounter += count + droppedWordsCounter.add(count) false } else { true diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala index 918e124065e4..787bbec73b28 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala @@ -19,9 +19,8 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf -import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext, Time} @@ -60,19 +59,19 @@ object SqlNetworkWordCount { // Convert RDDs of the words DStream to DataFrame and run SQL query words.foreachRDD { (rdd: RDD[String], time: Time) => - // Get the singleton instance of SQLContext - val sqlContext = SQLContextSingleton.getInstance(rdd.sparkContext) - import sqlContext.implicits._ + // Get the singleton instance of SparkSession + val spark = SparkSessionSingleton.getInstance(rdd.sparkContext.getConf) + import spark.implicits._ // Convert RDD[String] to RDD[case class] to DataFrame val wordsDataFrame = rdd.map(w => Record(w)).toDF() - // Register as table - wordsDataFrame.registerTempTable("words") + // Creates a temporary view using the DataFrame + wordsDataFrame.createOrReplaceTempView("words") // Do word count on table using SQL and print it val wordCountsDataFrame = - sqlContext.sql("select word, count(*) as total from words group by word") + spark.sql("select word, count(*) as total from words group by word") println(s"========= $time =========") wordCountsDataFrame.show() } @@ -87,14 +86,17 @@ object SqlNetworkWordCount { case class Record(word: String) -/** Lazily instantiated singleton instance of SQLContext */ -object SQLContextSingleton { +/** Lazily instantiated singleton instance of SparkSession */ +object SparkSessionSingleton { - @transient private var instance: SQLContext = _ + @transient private var instance: SparkSession = _ - def getInstance(sparkContext: SparkContext): SQLContext = { + def getInstance(sparkConf: SparkConf): SparkSession = { if (instance == null) { - instance = new SQLContext(sparkContext) + instance = SparkSession + .builder + .config(sparkConf) + .getOrCreate() } instance } diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 53a24f3e06e0..21d40863b77f 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -85,13 +85,11 @@ org.apache.httpcomponents httpclient - 4.5 test org.apache.httpcomponents httpcore - 4.4.1 test @@ -128,7 +126,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} ${project.version} test diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala index 4fe1ef669720..3da34b1b382d 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala @@ -34,10 +34,10 @@ class DB2IntegrationSuite extends DockerJDBCIntegrationSuite { "DB2INST1_PASSWORD" -> "rootpass", "LICENSE" -> "accept" ) - override val usesIpc = true + override val usesIpc = false override val jdbcPort: Int = 50000 override def getJdbcUrl(ip: String, port: Int): String = - s"jdbc:db2://$ip:$port/foo:user=db2inst1;password=rootpass;" + s"jdbc:db2://$ip:$port/foo:user=db2inst1;password=rootpass;retrieveMessagesFromServerOnGetMessage=true;" //scalastyle:ignore override def getStartupProcessName: Option[String] = Some("db2start") } @@ -46,22 +46,19 @@ class DB2IntegrationSuite extends DockerJDBCIntegrationSuite { conn.prepareStatement("INSERT INTO tbl VALUES (42,'fred')").executeUpdate() conn.prepareStatement("INSERT INTO tbl VALUES (17,'dave')").executeUpdate() - conn.prepareStatement("CREATE TABLE numbers (onebit BIT(1), tenbits BIT(10), " - + "small SMALLINT, med MEDIUMINT, nor INT, big BIGINT, deci DECIMAL(40,20), flt FLOAT, " - + "dbl DOUBLE)").executeUpdate() - conn.prepareStatement("INSERT INTO numbers VALUES (b'0', b'1000100101', " - + "17, 77777, 123456789, 123456789012345, 123456789012345.123456789012345, " - + "42.75, 1.0000000000000002)").executeUpdate() + conn.prepareStatement("CREATE TABLE numbers ( small SMALLINT, med INTEGER, big BIGINT, " + + "deci DECIMAL(31,20), flt FLOAT, dbl DOUBLE)").executeUpdate() + conn.prepareStatement("INSERT INTO numbers VALUES (17, 77777, 922337203685477580, " + + "123456745.56789012345000000000, 42.75, 5.4E-70)").executeUpdate() - conn.prepareStatement("CREATE TABLE dates (d DATE, t TIME, dt DATETIME, ts TIMESTAMP, " - + "yr YEAR)").executeUpdate() + conn.prepareStatement("CREATE TABLE dates (d DATE, t TIME, ts TIMESTAMP )").executeUpdate() conn.prepareStatement("INSERT INTO dates VALUES ('1991-11-09', '13:31:24', " - + "'1996-01-01 01:23:45', '2009-02-13 23:31:30', '2001')").executeUpdate() + + "'2009-02-13 23:31:30')").executeUpdate() // TODO: Test locale conversion for strings. - conn.prepareStatement("CREATE TABLE strings (a CHAR(10), b VARCHAR(10), c CLOB, d BLOB, " - + "e CHAR FOR BIT DATA)").executeUpdate() - conn.prepareStatement("INSERT INTO strings VALUES ('the', 'quick', 'brown', 'fox', 'jumps'") + conn.prepareStatement("CREATE TABLE strings (a CHAR(10), b VARCHAR(10), c CLOB, d BLOB)") + .executeUpdate() + conn.prepareStatement("INSERT INTO strings VALUES ('the', 'quick', 'brown', BLOB('fox'))") .executeUpdate() } @@ -80,26 +77,20 @@ class DB2IntegrationSuite extends DockerJDBCIntegrationSuite { val rows = df.collect() assert(rows.length == 1) val types = rows(0).toSeq.map(x => x.getClass.toString) - assert(types.length == 9) - assert(types(0).equals("class java.lang.Boolean")) - assert(types(1).equals("class java.lang.Long")) - assert(types(2).equals("class java.lang.Integer")) - assert(types(3).equals("class java.lang.Integer")) - assert(types(4).equals("class java.lang.Integer")) - assert(types(5).equals("class java.lang.Long")) - assert(types(6).equals("class java.math.BigDecimal")) - assert(types(7).equals("class java.lang.Double")) - assert(types(8).equals("class java.lang.Double")) - assert(rows(0).getBoolean(0) == false) - assert(rows(0).getLong(1) == 0x225) - assert(rows(0).getInt(2) == 17) - assert(rows(0).getInt(3) == 77777) - assert(rows(0).getInt(4) == 123456789) - assert(rows(0).getLong(5) == 123456789012345L) - val bd = new BigDecimal("123456789012345.12345678901234500000") - assert(rows(0).getAs[BigDecimal](6).equals(bd)) - assert(rows(0).getDouble(7) == 42.75) - assert(rows(0).getDouble(8) == 1.0000000000000002) + assert(types.length == 6) + assert(types(0).equals("class java.lang.Integer")) + assert(types(1).equals("class java.lang.Integer")) + assert(types(2).equals("class java.lang.Long")) + assert(types(3).equals("class java.math.BigDecimal")) + assert(types(4).equals("class java.lang.Double")) + assert(types(5).equals("class java.lang.Double")) + assert(rows(0).getInt(0) == 17) + assert(rows(0).getInt(1) == 77777) + assert(rows(0).getLong(2) == 922337203685477580L) + val bd = new BigDecimal("123456745.56789012345000000000") + assert(rows(0).getAs[BigDecimal](3).equals(bd)) + assert(rows(0).getDouble(4) == 42.75) + assert(rows(0).getDouble(5) == 5.4E-70) } test("Date types") { @@ -107,17 +98,13 @@ class DB2IntegrationSuite extends DockerJDBCIntegrationSuite { val rows = df.collect() assert(rows.length == 1) val types = rows(0).toSeq.map(x => x.getClass.toString) - assert(types.length == 5) + assert(types.length == 3) assert(types(0).equals("class java.sql.Date")) assert(types(1).equals("class java.sql.Timestamp")) assert(types(2).equals("class java.sql.Timestamp")) - assert(types(3).equals("class java.sql.Timestamp")) - assert(types(4).equals("class java.sql.Date")) assert(rows(0).getAs[Date](0).equals(Date.valueOf("1991-11-09"))) assert(rows(0).getAs[Timestamp](1).equals(Timestamp.valueOf("1970-01-01 13:31:24"))) - assert(rows(0).getAs[Timestamp](2).equals(Timestamp.valueOf("1996-01-01 01:23:45"))) - assert(rows(0).getAs[Timestamp](3).equals(Timestamp.valueOf("2009-02-13 23:31:30"))) - assert(rows(0).getAs[Date](4).equals(Date.valueOf("2001-01-01"))) + assert(rows(0).getAs[Timestamp](2).equals(Timestamp.valueOf("2009-02-13 23:31:30"))) } test("String types") { @@ -125,32 +112,22 @@ class DB2IntegrationSuite extends DockerJDBCIntegrationSuite { val rows = df.collect() assert(rows.length == 1) val types = rows(0).toSeq.map(x => x.getClass.toString) - assert(types.length == 9) + assert(types.length == 4) assert(types(0).equals("class java.lang.String")) assert(types(1).equals("class java.lang.String")) assert(types(2).equals("class java.lang.String")) - assert(types(3).equals("class java.lang.String")) - assert(types(4).equals("class java.lang.String")) - assert(types(5).equals("class java.lang.String")) - assert(types(6).equals("class [B")) - assert(types(7).equals("class [B")) - assert(types(8).equals("class [B")) - assert(rows(0).getString(0).equals("the")) + assert(types(3).equals("class [B")) + assert(rows(0).getString(0).equals("the ")) assert(rows(0).getString(1).equals("quick")) assert(rows(0).getString(2).equals("brown")) - assert(rows(0).getString(3).equals("fox")) - assert(rows(0).getString(4).equals("jumps")) - assert(rows(0).getString(5).equals("over")) - assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](6), Array[Byte](116, 104, 101, 0))) - assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](7), Array[Byte](108, 97, 122, 121))) - assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](8), Array[Byte](100, 111, 103))) + assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](3), Array[Byte](102, 111, 120))) } test("Basic write test") { - val df1 = sqlContext.read.jdbc(jdbcUrl, "numbers", new Properties) + // val df1 = sqlContext.read.jdbc(jdbcUrl, "numbers", new Properties) val df2 = sqlContext.read.jdbc(jdbcUrl, "dates", new Properties) val df3 = sqlContext.read.jdbc(jdbcUrl, "strings", new Properties) - df1.write.jdbc(jdbcUrl, "numberscopy", new Properties) + // df1.write.jdbc(jdbcUrl, "numberscopy", new Properties) df2.write.jdbc(jdbcUrl, "datescopy", new Properties) df3.write.jdbc(jdbcUrl, "stringscopy", new Properties) } diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala index aa47228eff3a..a70ed98b52d5 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala @@ -21,12 +21,9 @@ import java.math.BigDecimal import java.sql.{Connection, Date, Timestamp} import java.util.Properties -import org.scalatest.Ignore - import org.apache.spark.tags.DockerTest @DockerTest -@Ignore class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { override val imageName = "mysql:5.7.9" diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 357866b87ca2..c5e1f8607b33 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -20,8 +20,6 @@ package org.apache.spark.sql.jdbc import java.sql.Connection import java.util.Properties -import org.scalatest.Ignore - import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.tags.DockerTest @@ -46,12 +44,11 @@ import org.apache.spark.tags.DockerTest * repository. */ @DockerTest -@Ignore class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLContext { import testImplicits._ override val db = new DatabaseOnDocker { - override val imageName = "wnameless/oracle-xe-11g:latest" + override val imageName = "wnameless/oracle-xe-11g:14.04.4" override val env = Map( "ORACLE_ROOT_PASSWORD" -> "oracle" ) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index 6546d4cfd7ce..79dd70116ecb 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -20,15 +20,12 @@ package org.apache.spark.sql.jdbc import java.sql.Connection import java.util.Properties -import org.scalatest.Ignore - import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.types.{ArrayType, DecimalType} import org.apache.spark.tags.DockerTest @DockerTest -@Ignore class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { override val imageName = "postgres:9.4.5" diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index e4effe158c82..016af934bb89 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -92,7 +92,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/external/flume/pom.xml b/external/flume/pom.xml index d650dd034d63..f51d334de087 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -68,7 +68,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala index 6a4dafb8eddb..15ff4f60259f 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala @@ -116,7 +116,7 @@ private[flume] class PollingFlumeTestUtils { /** * Send data and wait until all data has been received */ - def sendDatAndEnsureAllDataHasBeenReceived(): Unit = { + def sendDataAndEnsureAllDataHasBeenReceived(): Unit = { val executor = Executors.newCachedThreadPool() val executorCompletion = new ExecutorCompletionService[Void](executor) @@ -174,7 +174,7 @@ private[flume] class PollingFlumeTestUtils { val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") queueRemaining.setAccessible(true) val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") - if (m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] != 5000) { + if (m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] != channelCapacity) { throw new AssertionError(s"Channel ${channel.getName} is not empty") } } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index 156712483d3a..1c93079497f6 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -24,10 +24,10 @@ import scala.collection.JavaConverters._ import scala.concurrent.duration._ import scala.language.postfixOps -import org.scalatest.BeforeAndAfter +import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.internal.Logging import org.apache.spark.network.util.JavaUtils import org.apache.spark.storage.StorageLevel @@ -35,11 +35,13 @@ import org.apache.spark.streaming.{Seconds, StreamingContext, TestOutputStream} import org.apache.spark.streaming.dstream.ReceiverInputDStream import org.apache.spark.util.{ManualClock, Utils} -class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfter with Logging { +class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { val maxAttempts = 5 val batchDuration = Seconds(1) + @transient private var _sc: SparkContext = _ + val conf = new SparkConf() .setMaster("local[2]") .setAppName(this.getClass.getSimpleName) @@ -47,6 +49,17 @@ class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfter with Log val utils = new PollingFlumeTestUtils + override def beforeAll(): Unit = { + _sc = new SparkContext(conf) + } + + override def afterAll(): Unit = { + if (_sc != null) { + _sc.stop() + _sc = null + } + } + test("flume polling test") { testMultipleTimes(testFlumePolling) } @@ -98,7 +111,7 @@ class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfter with Log def writeAndVerify(sinkPorts: Seq[Int]): Unit = { // Set up the streaming context and input streams - val ssc = new StreamingContext(conf, batchDuration) + val ssc = new StreamingContext(_sc, batchDuration) val addresses = sinkPorts.map(port => new InetSocketAddress("localhost", port)) val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, @@ -109,7 +122,7 @@ class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfter with Log ssc.start() try { - utils.sendDatAndEnsureAllDataHasBeenReceived() + utils.sendDataAndEnsureAllDataHasBeenReceived() val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] clock.advance(batchDuration.milliseconds) @@ -123,7 +136,8 @@ class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfter with Log utils.assertOutput(headers.asJava, bodies.asJava) } } finally { - ssc.stop() + // here stop ssc only, but not underlying sparkcontext + ssc.stop(false) } } diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 1ea9196e9dfe..60e3ff60df06 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -72,7 +72,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/external/java8-tests/src/test/java/org/apache/spark/sql/Java8DatasetAggregatorSuite.java b/external/java8-tests/src/test/java/org/apache/spark/sql/Java8DatasetAggregatorSuite.java index 23abfa397061..1a2aea67d9d0 100644 --- a/external/java8-tests/src/test/java/org/apache/spark/sql/Java8DatasetAggregatorSuite.java +++ b/external/java8-tests/src/test/java/org/apache/spark/sql/Java8DatasetAggregatorSuite.java @@ -25,7 +25,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.KeyValueGroupedDataset; -import org.apache.spark.sql.expressions.java.typed; +import org.apache.spark.sql.expressions.javalang.typed; /** * Suite that replicates tests in JavaDatasetAggregatorSuite using lambda syntax. diff --git a/external/kafka-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml similarity index 92% rename from external/kafka-assembly/pom.xml rename to external/kafka-0-8-assembly/pom.xml index 62818f5e8f43..3cc288abeaa2 100644 --- a/external/kafka-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -26,19 +26,19 @@ org.apache.spark - spark-streaming-kafka-assembly_2.11 + spark-streaming-kafka-0-8-assembly_2.11 jar Spark Project External Kafka Assembly http://spark.apache.org/ - streaming-kafka-assembly + streaming-kafka-0-8-assembly org.apache.spark - spark-streaming-kafka_${scala.binary.version} + spark-streaming-kafka-0-8_${scala.binary.version} ${project.version} @@ -65,16 +65,6 @@ protobuf-java provided - - com.sun.jersey - jersey-server - provided - - - com.sun.jersey - jersey-core - provided - net.jpountz.lz4 lz4 diff --git a/external/kafka/pom.xml b/external/kafka-0-8/pom.xml similarity index 93% rename from external/kafka/pom.xml rename to external/kafka-0-8/pom.xml index 68d52e9339b3..4a20b78917ef 100644 --- a/external/kafka/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -26,12 +26,12 @@ org.apache.spark - spark-streaming-kafka_2.11 + spark-streaming-kafka-0-8_2.11 - streaming-kafka + streaming-kafka-0-8 jar - Spark Project External Kafka + Spark Integration for Kafka 0.8 http://spark.apache.org/ @@ -88,7 +88,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala similarity index 100% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala similarity index 100% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala similarity index 100% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala similarity index 100% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala similarity index 100% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala similarity index 100% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala similarity index 100% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala similarity index 100% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala similarity index 100% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala similarity index 100% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package-info.java b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package-info.java similarity index 100% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package-info.java rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package-info.java diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package.scala similarity index 100% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package.scala diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java similarity index 100% rename from external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java rename to external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java similarity index 100% rename from external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java rename to external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java similarity index 100% rename from external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java rename to external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka-0-8/src/test/resources/log4j.properties similarity index 100% rename from external/kafka/src/test/resources/log4j.properties rename to external/kafka-0-8/src/test/resources/log4j.properties diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala similarity index 95% rename from external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala rename to external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index f14ff6705fd9..cb782d27fe22 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -281,14 +281,20 @@ class DirectKafkaStreamSuite sendDataAndWaitForReceive(i) } + ssc.stop() + // Verify that offset ranges were generated - val offsetRangesBeforeStop = getOffsetRanges(kafkaStream) - assert(offsetRangesBeforeStop.size >= 1, "No offset ranges generated") + // Since "offsetRangesAfterStop" will be used to compare with "recoveredOffsetRanges", we should + // collect offset ranges after stopping. Otherwise, because new RDDs keep being generated before + // stopping, we may not be able to get the latest RDDs, then "recoveredOffsetRanges" will + // contain something not in "offsetRangesAfterStop". + val offsetRangesAfterStop = getOffsetRanges(kafkaStream) + assert(offsetRangesAfterStop.size >= 1, "No offset ranges generated") assert( - offsetRangesBeforeStop.head._2.forall { _.fromOffset === 0 }, + offsetRangesAfterStop.head._2.forall { _.fromOffset === 0 }, "starting offset not zero" ) - ssc.stop() + logInfo("====== RESTARTING ========") // Recover context from checkpoints @@ -298,12 +304,14 @@ class DirectKafkaStreamSuite // Verify offset ranges have been recovered val recoveredOffsetRanges = getOffsetRanges(recoveredStream) assert(recoveredOffsetRanges.size > 0, "No offset ranges recovered") - val earlierOffsetRangesAsSets = offsetRangesBeforeStop.map { x => (x._1, x._2.toSet) } + val earlierOffsetRangesAsSets = offsetRangesAfterStop.map { x => (x._1, x._2.toSet) } assert( recoveredOffsetRanges.forall { or => earlierOffsetRangesAsSets.contains((or._1, or._2.toSet)) }, - "Recovered ranges are not the same as the ones generated" + "Recovered ranges are not the same as the ones generated\n" + + s"recoveredOffsetRanges: $recoveredOffsetRanges\n" + + s"earlierOffsetRangesAsSets: $earlierOffsetRangesAsSets" ) // Restart context, give more data and verify the total at the end // If the total is write that means each records has been received only once diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala similarity index 100% rename from external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala rename to external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala similarity index 100% rename from external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala rename to external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala similarity index 100% rename from external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala rename to external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala similarity index 100% rename from external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala rename to external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index d1c38c7ca5d6..6fb88ebae5b3 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -63,16 +63,26 @@ com.google.protobuf protobuf-java + 2.6.1 + + + + org.glassfish.jersey.core + jersey-client provided - com.sun.jersey - jersey-server + org.glassfish.jersey.core + jersey-common provided - com.sun.jersey - jersey-core + org.glassfish.jersey.core + jersey-server provided @@ -142,6 +152,15 @@ *:* + + + com.google.protobuf + kinesis.protobuf + + com.google.protobuf.** + + + *:* diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 935155eb5d36..b5f5ff2854cf 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -77,7 +77,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/graphx/pom.xml b/graphx/pom.xml index 1813f383cdcb..fc6c700dd1ec 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -72,7 +72,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index 80c6b6838faf..4da1ecb2a9af 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -209,7 +209,6 @@ object GraphGenerators extends Logging { } } - // TODO(crankshaw) turn result into an enum (or case class for pattern matching} private def pickQuadrant(a: Double, b: Double, c: Double, d: Double): Int = { if (a + b + c + d != 1.0) { throw new IllegalArgumentException("R-MAT probability parameters sum to " + (a + b + c + d) diff --git a/launcher/pom.xml b/launcher/pom.xml index ef731948826e..e7303853e656 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -65,7 +65,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java index 91586aad7b70..62a22008d0d5 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java @@ -325,7 +325,7 @@ static void addPermGenSizeOpt(List cmd) { return; } for (String arg : cmd) { - if (arg.startsWith("-XX:MaxPermSize=")) { + if (arg.contains("-XX:MaxPermSize=")) { return; } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index a083f05a2a9f..08873f581123 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -66,6 +66,13 @@ public class SparkLauncher { /** Logger name to use when launching a child process. */ public static final String CHILD_PROCESS_LOGGER_NAME = "spark.launcher.childProcLoggerName"; + /** + * A special value for the resource that tells Spark to not try to process the app resource as a + * file. This is useful when the class being executed is added to the application using other + * means - for example, by adding jars using the package download feature. + */ + public static final String NO_RESOURCE = "spark-internal"; + /** * Maximum time (in ms) to wait for a child process to connect back to the launcher server * when using @link{#start()}. diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index 6941ca903cd0..76897c4f7519 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -83,9 +83,9 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { static { specialClasses.put("org.apache.spark.repl.Main", "spark-shell"); specialClasses.put("org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver", - "spark-internal"); + SparkLauncher.NO_RESOURCE); specialClasses.put("org.apache.spark.sql.hive.thriftserver.HiveThriftServer2", - "spark-internal"); + SparkLauncher.NO_RESOURCE); } final List sparkArgs; @@ -112,11 +112,11 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { List submitArgs = args; if (args.size() > 0 && args.get(0).equals(PYSPARK_SHELL)) { this.allowsMixedArguments = true; - appResource = PYSPARK_SHELL_RESOURCE; + appResource = PYSPARK_SHELL; submitArgs = args.subList(1, args.size()); } else if (args.size() > 0 && args.get(0).equals(SPARKR_SHELL)) { this.allowsMixedArguments = true; - appResource = SPARKR_SHELL_RESOURCE; + appResource = SPARKR_SHELL; submitArgs = args.subList(1, args.size()); } else if (args.size() > 0 && args.get(0).equals(RUN_EXAMPLE)) { isExample = true; @@ -134,9 +134,9 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { @Override public List buildCommand(Map env) throws IOException, IllegalArgumentException { - if (PYSPARK_SHELL_RESOURCE.equals(appResource) && !printInfo) { + if (PYSPARK_SHELL.equals(appResource) && !printInfo) { return buildPySparkShellCommand(env); - } else if (SPARKR_SHELL_RESOURCE.equals(appResource) && !printInfo) { + } else if (SPARKR_SHELL.equals(appResource) && !printInfo) { return buildSparkRCommand(env); } else { return buildSparkSubmitCommand(env); @@ -147,6 +147,10 @@ List buildSparkSubmitArgs() { List args = new ArrayList<>(); SparkSubmitOptionParser parser = new SparkSubmitOptionParser(); + if (!allowsMixedArguments) { + checkArgument(appResource != null, "Missing application resource."); + } + if (verbose) { args.add(parser.VERBOSE); } @@ -278,6 +282,7 @@ private List buildPySparkShellCommand(Map env) throws IO // When launching the pyspark shell, the spark-submit arguments should be stored in the // PYSPARK_SUBMIT_ARGS env variable. + appResource = PYSPARK_SHELL_RESOURCE; constructEnvVarArgs(env, "PYSPARK_SUBMIT_ARGS"); // The executable is the PYSPARK_DRIVER_PYTHON env variable set by the pyspark script, @@ -301,6 +306,7 @@ private List buildSparkRCommand(Map env) throws IOExcept } // When launching the SparkR shell, store the spark-submit arguments in the SPARKR_SUBMIT_ARGS // env variable. + appResource = SPARKR_SHELL_RESOURCE; constructEnvVarArgs(env, "SPARKR_SUBMIT_ARGS"); // Set shell.R as R_PROFILE_USER to load the SparkR package when the shell comes up. @@ -435,22 +441,19 @@ protected boolean handleUnknown(String opt) { className = EXAMPLE_CLASS_PREFIX + className; } mainClass = className; - appResource = "spark-internal"; + appResource = SparkLauncher.NO_RESOURCE; return false; } else { checkArgument(!opt.startsWith("-"), "Unrecognized option: %s", opt); - sparkArgs.add(opt); + checkState(appResource == null, "Found unrecognized argument but resource is already set."); + appResource = opt; return false; } } @Override protected void handleExtraArgs(List extra) { - if (isExample) { - appArgs.addAll(extra); - } else { - sparkArgs.addAll(extra); - } + appArgs.addAll(extra); } } diff --git a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java index 4fafc43ef293..caeeea5ec6dd 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java @@ -99,12 +99,48 @@ public void testJavaMajorVersion() { assertEquals(10, javaMajorVersion("10")); } - private void testOpt(String opts, List expected) { + @Test + public void testAddPermGenSizeOpt() { + List cmd = new ArrayList<>(); + + if (javaMajorVersion(System.getProperty("java.version")) > 7) { + // Does nothing in Java 8 + addPermGenSizeOpt(cmd); + assertEquals(0, cmd.size()); + cmd.clear(); + + } else { + addPermGenSizeOpt(cmd); + assertEquals(1, cmd.size()); + assertTrue(cmd.get(0).startsWith("-XX:MaxPermSize=")); + cmd.clear(); + + cmd.add("foo"); + addPermGenSizeOpt(cmd); + assertEquals(2, cmd.size()); + assertTrue(cmd.get(1).startsWith("-XX:MaxPermSize=")); + cmd.clear(); + + cmd.add("-XX:MaxPermSize=512m"); + addPermGenSizeOpt(cmd); + assertEquals(1, cmd.size()); + assertEquals("-XX:MaxPermSize=512m", cmd.get(0)); + cmd.clear(); + + cmd.add("'-XX:MaxPermSize=512m'"); + addPermGenSizeOpt(cmd); + assertEquals(1, cmd.size()); + assertEquals("'-XX:MaxPermSize=512m'", cmd.get(0)); + cmd.clear(); + } + } + + private static void testOpt(String opts, List expected) { assertEquals(String.format("test string failed to parse: [[ %s ]]", opts), expected, parseOptionString(opts)); } - private void testInvalidOpt(String opts) { + private static void testInvalidOpt(String opts) { try { parseOptionString(opts); fail("Expected exception for invalid option string."); diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index c7e8b2e03a9f..c16f46a36027 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -72,7 +72,8 @@ public void testCliParser() throws Exception { parser.CONF, "spark.randomOption=foo", parser.CONF, - SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH + "=/driverLibPath"); + SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH + "=/driverLibPath", + SparkLauncher.NO_RESOURCE); Map env = new HashMap<>(); List cmd = buildCommand(sparkSubmitArgs, env); @@ -109,7 +110,8 @@ public void testAlternateSyntaxParsing() throws Exception { List sparkSubmitArgs = Arrays.asList( parser.CLASS + "=org.my.Class", parser.MASTER + "=foo", - parser.DEPLOY_MODE + "=bar"); + parser.DEPLOY_MODE + "=bar", + SparkLauncher.NO_RESOURCE); List cmd = newCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); assertEquals("org.my.Class", findArgValue(cmd, parser.CLASS)); @@ -168,6 +170,11 @@ public void testExamplesRunner() throws Exception { assertEquals("42", cmd.get(cmd.size() - 1)); } + @Test(expected = IllegalArgumentException.class) + public void testMissingAppResource() { + new SparkSubmitCommandBuilder().buildSparkSubmitArgs(); + } + private void testCmdBuilder(boolean isDriver, boolean useDefaultPropertyFile) throws Exception { String deployMode = isDriver ? "client" : "cluster"; diff --git a/licenses/LICENSE-postgresql.txt b/licenses/LICENSE-postgresql.txt new file mode 100644 index 000000000000..515bf9af4d43 --- /dev/null +++ b/licenses/LICENSE-postgresql.txt @@ -0,0 +1,24 @@ +PostgreSQL Database Management System +(formerly known as Postgres, then as Postgres95) + +Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group + +Portions Copyright (c) 1994, The Regents of the University of California + +Permission to use, copy, modify, and distribute this software and its +documentation for any purpose, without fee, and without a written agreement +is hereby granted, provided that the above copyright notice and this +paragraph and the following two paragraphs appear in all copies. + +IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR +DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING +LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS +DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. + +THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY +AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS +ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO +PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 68f15dd90502..1c6ab2b62d8f 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -53,6 +53,10 @@ mockito-core test + + org.apache.spark + spark-tags_${scala.binary.version} + diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala index 8204b5af02cf..a47526d36f1a 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala @@ -24,21 +24,28 @@ import scala.collection.mutable.{ArrayBuffer, ArrayBuilder => MArrayBuilder, Has import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM} import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.apache.spark.annotation.Since + /** * Trait for a local matrix. */ +@Since("2.0.0") sealed trait Matrix extends Serializable { /** Number of rows. */ + @Since("2.0.0") def numRows: Int /** Number of columns. */ + @Since("2.0.0") def numCols: Int /** Flag that keeps track whether the matrix is transposed or not. False by default. */ + @Since("2.0.0") val isTransposed: Boolean = false /** Converts to a dense array in column major. */ + @Since("2.0.0") def toArray: Array[Double] = { val newArray = new Array[Double](numRows * numCols) foreachActive { (i, j, v) => @@ -51,18 +58,21 @@ sealed trait Matrix extends Serializable { * Returns an iterator of column vectors. * This operation could be expensive, depending on the underlying storage. */ + @Since("2.0.0") def colIter: Iterator[Vector] /** * Returns an iterator of row vectors. * This operation could be expensive, depending on the underlying storage. */ + @Since("2.0.0") def rowIter: Iterator[Vector] = this.transpose.colIter /** Converts to a breeze matrix. */ private[ml] def toBreeze: BM[Double] /** Gets the (i, j)-th element. */ + @Since("2.0.0") def apply(i: Int, j: Int): Double /** Return the index for the (i, j)-th element in the backing array. */ @@ -72,12 +82,15 @@ sealed trait Matrix extends Serializable { private[ml] def update(i: Int, j: Int, v: Double): Unit /** Get a deep copy of the matrix. */ + @Since("2.0.0") def copy: Matrix /** Transpose the Matrix. Returns a new `Matrix` instance sharing the same underlying data. */ + @Since("2.0.0") def transpose: Matrix /** Convenience method for `Matrix`-`DenseMatrix` multiplication. */ + @Since("2.0.0") def multiply(y: DenseMatrix): DenseMatrix = { val C: DenseMatrix = DenseMatrix.zeros(numRows, y.numCols) BLAS.gemm(1.0, this, y, 0.0, C) @@ -85,11 +98,13 @@ sealed trait Matrix extends Serializable { } /** Convenience method for `Matrix`-`DenseVector` multiplication. For binary compatibility. */ + @Since("2.0.0") def multiply(y: DenseVector): DenseVector = { multiply(y.asInstanceOf[Vector]) } /** Convenience method for `Matrix`-`Vector` multiplication. */ + @Since("2.0.0") def multiply(y: Vector): DenseVector = { val output = new DenseVector(new Array[Double](numRows)) BLAS.gemv(1.0, this, y, 0.0, output) @@ -100,6 +115,7 @@ sealed trait Matrix extends Serializable { override def toString: String = toBreeze.toString() /** A human readable representation of the matrix with maximum lines and width */ + @Since("2.0.0") def toString(maxLines: Int, maxLineWidth: Int): String = toBreeze.toString(maxLines, maxLineWidth) /** @@ -129,11 +145,13 @@ sealed trait Matrix extends Serializable { /** * Find the number of non-zero active values. */ + @Since("2.0.0") def numNonzeros: Int /** * Find the number of values stored explicitly. These values can be zero as well. */ + @Since("2.0.0") def numActives: Int } @@ -154,10 +172,11 @@ sealed trait Matrix extends Serializable { * @param isTransposed whether the matrix is transposed. If true, `values` stores the matrix in * row major. */ -class DenseMatrix ( - val numRows: Int, - val numCols: Int, - val values: Array[Double], +@Since("2.0.0") +class DenseMatrix @Since("2.0.0") ( + @Since("2.0.0") val numRows: Int, + @Since("2.0.0") val numCols: Int, + @Since("2.0.0") val values: Array[Double], override val isTransposed: Boolean) extends Matrix { require(values.length == numRows * numCols, "The number of values supplied doesn't match the " + @@ -178,6 +197,7 @@ class DenseMatrix ( * @param numCols number of columns * @param values matrix entries in column major */ + @Since("2.0.0") def this(numRows: Int, numCols: Int, values: Array[Double]) = this(numRows, numCols, values, false) @@ -266,6 +286,7 @@ class DenseMatrix ( * Generate a `SparseMatrix` from the given `DenseMatrix`. The new matrix will have isTransposed * set to false. */ + @Since("2.0.0") def toSparse: SparseMatrix = { val spVals: MArrayBuilder[Double] = new MArrayBuilder.ofDouble val colPtrs: Array[Int] = new Array[Int](numCols + 1) @@ -307,6 +328,7 @@ class DenseMatrix ( /** * Factory methods for [[org.apache.spark.ml.linalg.DenseMatrix]]. */ +@Since("2.0.0") object DenseMatrix { /** @@ -315,6 +337,7 @@ object DenseMatrix { * @param numCols number of columns of the matrix * @return `DenseMatrix` with size `numRows` x `numCols` and values of zeros */ + @Since("2.0.0") def zeros(numRows: Int, numCols: Int): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -327,6 +350,7 @@ object DenseMatrix { * @param numCols number of columns of the matrix * @return `DenseMatrix` with size `numRows` x `numCols` and values of ones */ + @Since("2.0.0") def ones(numRows: Int, numCols: Int): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -338,6 +362,7 @@ object DenseMatrix { * @param n number of rows and columns of the matrix * @return `DenseMatrix` with size `n` x `n` and values of ones on the diagonal */ + @Since("2.0.0") def eye(n: Int): DenseMatrix = { val identity = DenseMatrix.zeros(n, n) var i = 0 @@ -355,6 +380,7 @@ object DenseMatrix { * @param rng a random number generator * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1) */ + @Since("2.0.0") def rand(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -368,6 +394,7 @@ object DenseMatrix { * @param rng a random number generator * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1) */ + @Since("2.0.0") def randn(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -380,6 +407,7 @@ object DenseMatrix { * @return Square `DenseMatrix` with size `values.length` x `values.length` and `values` * on the diagonal */ + @Since("2.0.0") def diag(vector: Vector): DenseMatrix = { val n = vector.size val matrix = DenseMatrix.zeros(n, n) @@ -415,12 +443,13 @@ object DenseMatrix { * Compressed Sparse Row (CSR) format, where `colPtrs` behaves as rowPtrs, * and `rowIndices` behave as colIndices, and `values` are stored in row major. */ -class SparseMatrix ( - val numRows: Int, - val numCols: Int, - val colPtrs: Array[Int], - val rowIndices: Array[Int], - val values: Array[Double], +@Since("2.0.0") +class SparseMatrix @Since("2.0.0") ( + @Since("2.0.0") val numRows: Int, + @Since("2.0.0") val numCols: Int, + @Since("2.0.0") val colPtrs: Array[Int], + @Since("2.0.0") val rowIndices: Array[Int], + @Since("2.0.0") val values: Array[Double], override val isTransposed: Boolean) extends Matrix { require(values.length == rowIndices.length, "The number of row indices and values don't match! " + @@ -451,6 +480,7 @@ class SparseMatrix ( * order for each column * @param values non-zero matrix entries in column major */ + @Since("2.0.0") def this( numRows: Int, numCols: Int, @@ -550,6 +580,7 @@ class SparseMatrix ( * Generate a `DenseMatrix` from the given `SparseMatrix`. The new matrix will have isTransposed * set to false. */ + @Since("2.0.0") def toDense: DenseMatrix = { new DenseMatrix(numRows, numCols, toArray) } @@ -594,6 +625,7 @@ class SparseMatrix ( /** * Factory methods for [[org.apache.spark.ml.linalg.SparseMatrix]]. */ +@Since("2.0.0") object SparseMatrix { /** @@ -605,6 +637,7 @@ object SparseMatrix { * @param entries Array of (i, j, value) tuples * @return The corresponding `SparseMatrix` */ + @Since("2.0.0") def fromCOO(numRows: Int, numCols: Int, entries: Iterable[(Int, Int, Double)]): SparseMatrix = { val sortedEntries = entries.toSeq.sortBy(v => (v._2, v._1)) val numEntries = sortedEntries.size @@ -653,6 +686,7 @@ object SparseMatrix { * @param n number of rows and columns of the matrix * @return `SparseMatrix` with size `n` x `n` and values of ones on the diagonal */ + @Since("2.0.0") def speye(n: Int): SparseMatrix = { new SparseMatrix(n, n, (0 to n).toArray, (0 until n).toArray, Array.fill(n)(1.0)) } @@ -722,6 +756,7 @@ object SparseMatrix { * @param rng a random number generator * @return `SparseMatrix` with size `numRows` x `numCols` and values in U(0, 1) */ + @Since("2.0.0") def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = { val mat = genRandMatrix(numRows, numCols, density, rng) mat.update(i => rng.nextDouble()) @@ -735,6 +770,7 @@ object SparseMatrix { * @param rng a random number generator * @return `SparseMatrix` with size `numRows` x `numCols` and values in N(0, 1) */ + @Since("2.0.0") def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = { val mat = genRandMatrix(numRows, numCols, density, rng) mat.update(i => rng.nextGaussian()) @@ -746,6 +782,7 @@ object SparseMatrix { * @return Square `SparseMatrix` with size `values.length` x `values.length` and non-zero * `values` on the diagonal */ + @Since("2.0.0") def spdiag(vector: Vector): SparseMatrix = { val n = vector.size vector match { @@ -762,6 +799,7 @@ object SparseMatrix { /** * Factory methods for [[org.apache.spark.ml.linalg.Matrix]]. */ +@Since("2.0.0") object Matrices { /** @@ -771,6 +809,7 @@ object Matrices { * @param numCols number of columns * @param values matrix entries in column major */ + @Since("2.0.0") def dense(numRows: Int, numCols: Int, values: Array[Double]): Matrix = { new DenseMatrix(numRows, numCols, values) } @@ -784,6 +823,7 @@ object Matrices { * @param rowIndices the row index of the entry * @param values non-zero matrix entries in column major */ + @Since("2.0.0") def sparse( numRows: Int, numCols: Int, @@ -825,6 +865,7 @@ object Matrices { * @param numCols number of columns of the matrix * @return `Matrix` with size `numRows` x `numCols` and values of zeros */ + @Since("2.0.0") def zeros(numRows: Int, numCols: Int): Matrix = DenseMatrix.zeros(numRows, numCols) /** @@ -833,6 +874,7 @@ object Matrices { * @param numCols number of columns of the matrix * @return `Matrix` with size `numRows` x `numCols` and values of ones */ + @Since("2.0.0") def ones(numRows: Int, numCols: Int): Matrix = DenseMatrix.ones(numRows, numCols) /** @@ -840,6 +882,7 @@ object Matrices { * @param n number of rows and columns of the matrix * @return `Matrix` with size `n` x `n` and values of ones on the diagonal */ + @Since("2.0.0") def eye(n: Int): Matrix = DenseMatrix.eye(n) /** @@ -847,6 +890,7 @@ object Matrices { * @param n number of rows and columns of the matrix * @return `Matrix` with size `n` x `n` and values of ones on the diagonal */ + @Since("2.0.0") def speye(n: Int): Matrix = SparseMatrix.speye(n) /** @@ -856,6 +900,7 @@ object Matrices { * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1) */ + @Since("2.0.0") def rand(numRows: Int, numCols: Int, rng: Random): Matrix = DenseMatrix.rand(numRows, numCols, rng) @@ -867,6 +912,7 @@ object Matrices { * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1) */ + @Since("2.0.0") def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix = SparseMatrix.sprand(numRows, numCols, density, rng) @@ -877,6 +923,7 @@ object Matrices { * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1) */ + @Since("2.0.0") def randn(numRows: Int, numCols: Int, rng: Random): Matrix = DenseMatrix.randn(numRows, numCols, rng) @@ -888,6 +935,7 @@ object Matrices { * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1) */ + @Since("2.0.0") def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix = SparseMatrix.sprandn(numRows, numCols, density, rng) @@ -897,6 +945,7 @@ object Matrices { * @return Square `Matrix` with size `values.length` x `values.length` and `values` * on the diagonal */ + @Since("2.0.0") def diag(vector: Vector): Matrix = DenseMatrix.diag(vector) /** @@ -906,6 +955,7 @@ object Matrices { * @param matrices array of matrices * @return a single `Matrix` composed of the matrices that were horizontally concatenated */ + @Since("2.0.0") def horzcat(matrices: Array[Matrix]): Matrix = { if (matrices.isEmpty) { return new DenseMatrix(0, 0, Array[Double]()) @@ -964,6 +1014,7 @@ object Matrices { * @param matrices array of matrices * @return a single `Matrix` composed of the matrices that were vertically concatenated */ + @Since("2.0.0") def vertcat(matrices: Array[Matrix]): Matrix = { if (matrices.isEmpty) { return new DenseMatrix(0, 0, Array[Double]()) diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala index c0d112d2c53d..59f9c2adba5f 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala @@ -25,21 +25,26 @@ import scala.collection.JavaConverters._ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} +import org.apache.spark.annotation.Since + /** * Represents a numeric vector, whose index type is Int and value type is Double. * * Note: Users should not implement this interface. */ +@Since("2.0.0") sealed trait Vector extends Serializable { /** * Size of the vector. */ + @Since("2.0.0") def size: Int /** * Converts the instance to a double array. */ + @Since("2.0.0") def toArray: Array[Double] override def equals(other: Any): Boolean = { @@ -93,11 +98,13 @@ sealed trait Vector extends Serializable { * Gets the value of the ith element. * @param i index */ + @Since("2.0.0") def apply(i: Int): Double = toBreeze(i) /** * Makes a deep copy of this vector. */ + @Since("2.0.0") def copy: Vector = { throw new NotImplementedError(s"copy is not implemented for ${this.getClass}.") } @@ -109,32 +116,38 @@ sealed trait Vector extends Serializable { * the vector with type `Int`, and the second parameter is the corresponding value * with type `Double`. */ + @Since("2.0.0") def foreachActive(f: (Int, Double) => Unit): Unit /** * Number of active entries. An "active entry" is an element which is explicitly stored, * regardless of its value. Note that inactive entries have value 0. */ + @Since("2.0.0") def numActives: Int /** * Number of nonzero elements. This scans all active values and count nonzeros. */ + @Since("2.0.0") def numNonzeros: Int /** * Converts this vector to a sparse vector with all explicit zeros removed. */ + @Since("2.0.0") def toSparse: SparseVector /** * Converts this vector to a dense vector. */ + @Since("2.0.0") def toDense: DenseVector = new DenseVector(this.toArray) /** * Returns a vector in either dense or sparse format, whichever uses less storage. */ + @Since("2.0.0") def compressed: Vector = { val nnz = numNonzeros // A dense vector needs 8 * size + 8 bytes, while a sparse vector needs 12 * nnz + 20 bytes. @@ -149,6 +162,7 @@ sealed trait Vector extends Serializable { * Find the index of a maximal element. Returns the first maximal element in case of a tie. * Returns -1 if vector has length 0. */ + @Since("2.0.0") def argmax: Int } @@ -157,12 +171,14 @@ sealed trait Vector extends Serializable { * We don't use the name `Vector` because Scala imports * [[scala.collection.immutable.Vector]] by default. */ +@Since("2.0.0") object Vectors { /** * Creates a dense vector from its values. */ @varargs + @Since("2.0.0") def dense(firstValue: Double, otherValues: Double*): Vector = new DenseVector((firstValue +: otherValues).toArray) @@ -170,6 +186,7 @@ object Vectors { /** * Creates a dense vector from a double array. */ + @Since("2.0.0") def dense(values: Array[Double]): Vector = new DenseVector(values) /** @@ -179,6 +196,7 @@ object Vectors { * @param indices index array, must be strictly increasing. * @param values value array, must have the same length as indices. */ + @Since("2.0.0") def sparse(size: Int, indices: Array[Int], values: Array[Double]): Vector = new SparseVector(size, indices, values) @@ -188,6 +206,7 @@ object Vectors { * @param size vector size. * @param elements vector elements in (index, value) pairs. */ + @Since("2.0.0") def sparse(size: Int, elements: Seq[(Int, Double)]): Vector = { require(size > 0, "The size of the requested sparse vector must be greater than 0.") @@ -209,6 +228,7 @@ object Vectors { * @param size vector size. * @param elements vector elements in (index, value) pairs. */ + @Since("2.0.0") def sparse(size: Int, elements: JavaIterable[(JavaInteger, JavaDouble)]): Vector = { sparse(size, elements.asScala.map { case (i, x) => (i.intValue(), x.doubleValue()) @@ -221,6 +241,7 @@ object Vectors { * @param size vector size * @return a zero vector */ + @Since("2.0.0") def zeros(size: Int): Vector = { new DenseVector(new Array[Double](size)) } @@ -253,6 +274,7 @@ object Vectors { * @param p norm. * @return norm in L^p^ space. */ + @Since("2.0.0") def norm(vector: Vector, p: Double): Double = { require(p >= 1.0, "To compute the p-norm of the vector, we require that you specify a p>=1. " + s"You specified p=$p.") @@ -305,6 +327,7 @@ object Vectors { * @param v2 second Vector. * @return squared distance between two Vectors. */ + @Since("2.0.0") def sqdist(v1: Vector, v2: Vector): Double = { require(v1.size == v2.size, s"Vector dimensions do not match: Dim(v1)=${v1.size} and Dim(v2)" + s"=${v2.size}.") @@ -421,7 +444,8 @@ object Vectors { /** * A dense vector represented by a value array. */ -class DenseVector (val values: Array[Double]) extends Vector { +@Since("2.0.0") +class DenseVector @Since("2.0.0") ( @Since("2.0.0") val values: Array[Double]) extends Vector { override def size: Int = values.length @@ -515,9 +539,11 @@ class DenseVector (val values: Array[Double]) extends Vector { } } +@Since("2.0.0") object DenseVector { /** Extracts the value array from a dense vector. */ + @Since("2.0.0") def unapply(dv: DenseVector): Option[Array[Double]] = Some(dv.values) } @@ -528,10 +554,11 @@ object DenseVector { * @param indices index array, assume to be strictly increasing. * @param values value array, must have the same length as the index array. */ -class SparseVector ( +@Since("2.0.0") +class SparseVector @Since("2.0.0") ( override val size: Int, - val indices: Array[Int], - val values: Array[Double]) extends Vector { + @Since("2.0.0") val indices: Array[Int], + @Since("2.0.0") val values: Array[Double]) extends Vector { require(indices.length == values.length, "Sparse vectors require that the dimension of the" + s" indices match the dimension of the values. You provided ${indices.length} indices and " + @@ -693,7 +720,9 @@ class SparseVector ( } } +@Since("2.0.0") object SparseVector { + @Since("2.0.0") def unapply(sv: SparseVector): Option[(Int, Array[Int], Array[Double])] = Some((sv.size, sv.indices, sv.values)) } diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala b/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala index c62a1eab2016..383d6d96e8d7 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala @@ -19,6 +19,7 @@ package org.apache.spark.ml.stat.distribution import breeze.linalg.{diag, eigSym, max, DenseMatrix => BDM, DenseVector => BDV, Vector => BV} +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.ml.impl.Utils import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} @@ -32,9 +33,11 @@ import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} * @param mean The mean vector of the distribution * @param cov The covariance matrix of the distribution */ -class MultivariateGaussian( - val mean: Vector, - val cov: Matrix) extends Serializable { +@Since("2.0.0") +@DeveloperApi +class MultivariateGaussian @Since("2.0.0") ( + @Since("2.0.0") val mean: Vector, + @Since("2.0.0") val cov: Matrix) extends Serializable { require(cov.numCols == cov.numRows, "Covariance matrix must be square") require(mean.size == cov.numCols, "Mean vector length must match covariance matrix size") @@ -56,6 +59,7 @@ class MultivariateGaussian( /** * Returns density of this multivariate Gaussian at given point, x */ + @Since("2.0.0") def pdf(x: Vector): Double = { pdf(x.toBreeze) } @@ -63,6 +67,7 @@ class MultivariateGaussian( /** * Returns the log-density of this multivariate Gaussian at given point, x */ + @Since("2.0.0") def logpdf(x: Vector): Double = { logpdf(x.toBreeze) } diff --git a/mllib/pom.xml b/mllib/pom.xml index 24d8274e2222..c324afb2a2d1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -116,7 +116,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/README b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/README new file mode 100755 index 000000000000..ec08a5080774 --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/README @@ -0,0 +1,12 @@ +Stopwords Corpus + +This corpus contains lists of stop words for several languages. These +are high-frequency grammatical words which are usually ignored in text +retrieval applications. + +They were obtained from: +http://anoncvs.postgresql.org/cvsweb.cgi/pgsql/src/backend/snowball/stopwords/ + +The English list has been augmented +https://github.com/nltk/nltk_data/issues/22 + diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/danish.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/danish.txt new file mode 100644 index 000000000000..ea9e2c4abe5b --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/danish.txt @@ -0,0 +1,94 @@ +og +i +jeg +det +at +en +den +til +er +som +på +de +med +han +af +for +ikke +der +var +mig +sig +men +et +har +om +vi +min +havde +ham +hun +nu +over +da +fra +du +ud +sin +dem +os +op +man +hans +hvor +eller +hvad +skal +selv +her +alle +vil +blev +kunne +ind +når +være +dog +noget +ville +jo +deres +efter +ned +skulle +denne +end +dette +mit +også +under +have +dig +anden +hende +mine +alt +meget +sit +sine +vor +mod +disse +hvis +din +nogle +hos +blive +mange +ad +bliver +hendes +været +thi +jer +sådan \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/dutch.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/dutch.txt new file mode 100644 index 000000000000..023cc2c939b2 --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/dutch.txt @@ -0,0 +1,101 @@ +de +en +van +ik +te +dat +die +in +een +hij +het +niet +zijn +is +was +op +aan +met +als +voor +had +er +maar +om +hem +dan +zou +of +wat +mijn +men +dit +zo +door +over +ze +zich +bij +ook +tot +je +mij +uit +der +daar +haar +naar +heb +hoe +heeft +hebben +deze +u +want +nog +zal +me +zij +nu +ge +geen +omdat +iets +worden +toch +al +waren +veel +meer +doen +toen +moet +ben +zonder +kan +hun +dus +alles +onder +ja +eens +hier +wie +werd +altijd +doch +wordt +wezen +kunnen +ons +zelf +tegen +na +reeds +wil +kon +niets +uw +iemand +geweest +andere \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/english.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/english.txt new file mode 100644 index 000000000000..d075cc0babc3 --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/english.txt @@ -0,0 +1,153 @@ +i +me +my +myself +we +our +ours +ourselves +you +your +yours +yourself +yourselves +he +him +his +himself +she +her +hers +herself +it +its +itself +they +them +their +theirs +themselves +what +which +who +whom +this +that +these +those +am +is +are +was +were +be +been +being +have +has +had +having +do +does +did +doing +a +an +the +and +but +if +or +because +as +until +while +of +at +by +for +with +about +against +between +into +through +during +before +after +above +below +to +from +up +down +in +out +on +off +over +under +again +further +then +once +here +there +when +where +why +how +all +any +both +each +few +more +most +other +some +such +no +nor +not +only +own +same +so +than +too +very +s +t +can +will +just +don +should +now +d +ll +m +o +re +ve +y +ain +aren +couldn +didn +doesn +hadn +hasn +haven +isn +ma +mightn +mustn +needn +shan +shouldn +wasn +weren +won +wouldn diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/finnish.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/finnish.txt new file mode 100644 index 000000000000..5b0eb10777d0 --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/finnish.txt @@ -0,0 +1,235 @@ +olla +olen +olet +on +olemme +olette +ovat +ole +oli +olisi +olisit +olisin +olisimme +olisitte +olisivat +olit +olin +olimme +olitte +olivat +ollut +olleet +en +et +ei +emme +ette +eivät +minä +minun +minut +minua +minussa +minusta +minuun +minulla +minulta +minulle +sinä +sinun +sinut +sinua +sinussa +sinusta +sinuun +sinulla +sinulta +sinulle +hän +hänen +hänet +häntä +hänessä +hänestä +häneen +hänellä +häneltä +hänelle +me +meidän +meidät +meitä +meissä +meistä +meihin +meillä +meiltä +meille +te +teidän +teidät +teitä +teissä +teistä +teihin +teillä +teiltä +teille +he +heidän +heidät +heitä +heissä +heistä +heihin +heillä +heiltä +heille +tämä +tämän +tätä +tässä +tästä +tähän +tallä +tältä +tälle +tänä +täksi +tuo +tuon +tuotä +tuossa +tuosta +tuohon +tuolla +tuolta +tuolle +tuona +tuoksi +se +sen +sitä +siinä +siitä +siihen +sillä +siltä +sille +sinä +siksi +nämä +näiden +näitä +näissä +näistä +näihin +näillä +näiltä +näille +näinä +näiksi +nuo +noiden +noita +noissa +noista +noihin +noilla +noilta +noille +noina +noiksi +ne +niiden +niitä +niissä +niistä +niihin +niillä +niiltä +niille +niinä +niiksi +kuka +kenen +kenet +ketä +kenessä +kenestä +keneen +kenellä +keneltä +kenelle +kenenä +keneksi +ketkä +keiden +ketkä +keitä +keissä +keistä +keihin +keillä +keiltä +keille +keinä +keiksi +mikä +minkä +minkä +mitä +missä +mistä +mihin +millä +miltä +mille +minä +miksi +mitkä +joka +jonka +jota +jossa +josta +johon +jolla +jolta +jolle +jona +joksi +jotka +joiden +joita +joissa +joista +joihin +joilla +joilta +joille +joina +joiksi +että +ja +jos +koska +kuin +mutta +niin +sekä +sillä +tai +vaan +vai +vaikka +kanssa +mukaan +noin +poikki +yli +kun +niin +nyt +itse \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/french.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/french.txt new file mode 100644 index 000000000000..94b8f8f39a3e --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/french.txt @@ -0,0 +1,155 @@ +au +aux +avec +ce +ces +dans +de +des +du +elle +en +et +eux +il +je +la +le +leur +lui +ma +mais +me +même +mes +moi +mon +ne +nos +notre +nous +on +ou +par +pas +pour +qu +que +qui +sa +se +ses +son +sur +ta +te +tes +toi +ton +tu +un +une +vos +votre +vous +c +d +j +l +à +m +n +s +t +y +été +étée +étées +étés +étant +étante +étants +étantes +suis +es +est +sommes +êtes +sont +serai +seras +sera +serons +serez +seront +serais +serait +serions +seriez +seraient +étais +était +étions +étiez +étaient +fus +fut +fûmes +fûtes +furent +sois +soit +soyons +soyez +soient +fusse +fusses +fût +fussions +fussiez +fussent +ayant +ayante +ayantes +ayants +eu +eue +eues +eus +ai +as +avons +avez +ont +aurai +auras +aura +aurons +aurez +auront +aurais +aurait +aurions +auriez +auraient +avais +avait +avions +aviez +avaient +eut +eûmes +eûtes +eurent +aie +aies +ait +ayons +ayez +aient +eusse +eusses +eût +eussions +eussiez +eussent \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/german.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/german.txt new file mode 100644 index 000000000000..7e65190f8ba2 --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/german.txt @@ -0,0 +1,231 @@ +aber +alle +allem +allen +aller +alles +als +also +am +an +ander +andere +anderem +anderen +anderer +anderes +anderm +andern +anderr +anders +auch +auf +aus +bei +bin +bis +bist +da +damit +dann +der +den +des +dem +die +das +daß +derselbe +derselben +denselben +desselben +demselben +dieselbe +dieselben +dasselbe +dazu +dein +deine +deinem +deinen +deiner +deines +denn +derer +dessen +dich +dir +du +dies +diese +diesem +diesen +dieser +dieses +doch +dort +durch +ein +eine +einem +einen +einer +eines +einig +einige +einigem +einigen +einiger +einiges +einmal +er +ihn +ihm +es +etwas +euer +eure +eurem +euren +eurer +eures +für +gegen +gewesen +hab +habe +haben +hat +hatte +hatten +hier +hin +hinter +ich +mich +mir +ihr +ihre +ihrem +ihren +ihrer +ihres +euch +im +in +indem +ins +ist +jede +jedem +jeden +jeder +jedes +jene +jenem +jenen +jener +jenes +jetzt +kann +kein +keine +keinem +keinen +keiner +keines +können +könnte +machen +man +manche +manchem +manchen +mancher +manches +mein +meine +meinem +meinen +meiner +meines +mit +muss +musste +nach +nicht +nichts +noch +nun +nur +ob +oder +ohne +sehr +sein +seine +seinem +seinen +seiner +seines +selbst +sich +sie +ihnen +sind +so +solche +solchem +solchen +solcher +solches +soll +sollte +sondern +sonst +über +um +und +uns +unse +unsem +unsen +unser +unses +unter +viel +vom +von +vor +während +war +waren +warst +was +weg +weil +weiter +welche +welchem +welchen +welcher +welches +wenn +werde +werden +wie +wieder +will +wir +wird +wirst +wo +wollen +wollte +würde +würden +zu +zum +zur +zwar +zwischen \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt new file mode 100644 index 000000000000..8d4543a0965d --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt @@ -0,0 +1,199 @@ +a +ahogy +ahol +aki +akik +akkor +alatt +által +általában +amely +amelyek +amelyekben +amelyeket +amelyet +amelynek +ami +amit +amolyan +amíg +amikor +át +abban +ahhoz +annak +arra +arról +az +azok +azon +azt +azzal +azért +aztán +azután +azonban +bár +be +belül +benne +cikk +cikkek +cikkeket +csak +de +e +eddig +egész +egy +egyes +egyetlen +egyéb +egyik +egyre +ekkor +el +elég +ellen +elõ +elõször +elõtt +elsõ +én +éppen +ebben +ehhez +emilyen +ennek +erre +ez +ezt +ezek +ezen +ezzel +ezért +és +fel +felé +hanem +hiszen +hogy +hogyan +igen +így +illetve +ill. +ill +ilyen +ilyenkor +ison +ismét +itt +jó +jól +jobban +kell +kellett +keresztül +keressünk +ki +kívül +között +közül +legalább +lehet +lehetett +legyen +lenne +lenni +lesz +lett +maga +magát +majd +majd +már +más +másik +meg +még +mellett +mert +mely +melyek +mi +mit +míg +miért +milyen +mikor +minden +mindent +mindenki +mindig +mint +mintha +mivel +most +nagy +nagyobb +nagyon +ne +néha +nekem +neki +nem +néhány +nélkül +nincs +olyan +ott +össze +õ +õk +õket +pedig +persze +rá +s +saját +sem +semmi +sok +sokat +sokkal +számára +szemben +szerint +szinte +talán +tehát +teljes +tovább +továbbá +több +úgy +ugyanis +új +újabb +újra +után +utána +utolsó +vagy +vagyis +valaki +valami +valamint +való +vagyok +van +vannak +volt +voltam +voltak +voltunk +vissza +vele +viszont +volna \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/italian.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/italian.txt new file mode 100644 index 000000000000..783b2e0cbfcd --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/italian.txt @@ -0,0 +1,279 @@ +ad +al +allo +ai +agli +all +agl +alla +alle +con +col +coi +da +dal +dallo +dai +dagli +dall +dagl +dalla +dalle +di +del +dello +dei +degli +dell +degl +della +delle +in +nel +nello +nei +negli +nell +negl +nella +nelle +su +sul +sullo +sui +sugli +sull +sugl +sulla +sulle +per +tra +contro +io +tu +lui +lei +noi +voi +loro +mio +mia +miei +mie +tuo +tua +tuoi +tue +suo +sua +suoi +sue +nostro +nostra +nostri +nostre +vostro +vostra +vostri +vostre +mi +ti +ci +vi +lo +la +li +le +gli +ne +il +un +uno +una +ma +ed +se +perché +anche +come +dov +dove +che +chi +cui +non +più +quale +quanto +quanti +quanta +quante +quello +quelli +quella +quelle +questo +questi +questa +queste +si +tutto +tutti +a +c +e +i +l +o +ho +hai +ha +abbiamo +avete +hanno +abbia +abbiate +abbiano +avrò +avrai +avrà +avremo +avrete +avranno +avrei +avresti +avrebbe +avremmo +avreste +avrebbero +avevo +avevi +aveva +avevamo +avevate +avevano +ebbi +avesti +ebbe +avemmo +aveste +ebbero +avessi +avesse +avessimo +avessero +avendo +avuto +avuta +avuti +avute +sono +sei +è +siamo +siete +sia +siate +siano +sarò +sarai +sarà +saremo +sarete +saranno +sarei +saresti +sarebbe +saremmo +sareste +sarebbero +ero +eri +era +eravamo +eravate +erano +fui +fosti +fu +fummo +foste +furono +fossi +fosse +fossimo +fossero +essendo +faccio +fai +facciamo +fanno +faccia +facciate +facciano +farò +farai +farà +faremo +farete +faranno +farei +faresti +farebbe +faremmo +fareste +farebbero +facevo +facevi +faceva +facevamo +facevate +facevano +feci +facesti +fece +facemmo +faceste +fecero +facessi +facesse +facessimo +facessero +facendo +sto +stai +sta +stiamo +stanno +stia +stiate +stiano +starò +starai +starà +staremo +starete +staranno +starei +staresti +starebbe +staremmo +stareste +starebbero +stavo +stavi +stava +stavamo +stavate +stavano +stetti +stesti +stette +stemmo +steste +stettero +stessi +stesse +stessimo +stessero +stando \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/norwegian.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/norwegian.txt new file mode 100644 index 000000000000..cb91702c5e9a --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/norwegian.txt @@ -0,0 +1,176 @@ +og +i +jeg +det +at +en +et +den +til +er +som +på +de +med +han +av +ikke +ikkje +der +så +var +meg +seg +men +ett +har +om +vi +min +mitt +ha +hadde +hun +nå +over +da +ved +fra +du +ut +sin +dem +oss +opp +man +kan +hans +hvor +eller +hva +skal +selv +sjøl +her +alle +vil +bli +ble +blei +blitt +kunne +inn +når +være +kom +noen +noe +ville +dere +som +deres +kun +ja +etter +ned +skulle +denne +for +deg +si +sine +sitt +mot +å +meget +hvorfor +dette +disse +uten +hvordan +ingen +din +ditt +blir +samme +hvilken +hvilke +sånn +inni +mellom +vår +hver +hvem +vors +hvis +både +bare +enn +fordi +før +mange +også +slik +vært +være +båe +begge +siden +dykk +dykkar +dei +deira +deires +deim +di +då +eg +ein +eit +eitt +elles +honom +hjå +ho +hoe +henne +hennar +hennes +hoss +hossen +ikkje +ingi +inkje +korleis +korso +kva +kvar +kvarhelst +kven +kvi +kvifor +me +medan +mi +mine +mykje +no +nokon +noka +nokor +noko +nokre +si +sia +sidan +so +somt +somme +um +upp +vere +vore +verte +vort +varte +vart \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt new file mode 100644 index 000000000000..98b4fdcdf7a2 --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt @@ -0,0 +1,203 @@ +de +a +o +que +e +do +da +em +um +para +com +não +uma +os +no +se +na +por +mais +as +dos +como +mas +ao +ele +das +à +seu +sua +ou +quando +muito +nos +já +eu +também +só +pelo +pela +até +isso +ela +entre +depois +sem +mesmo +aos +seus +quem +nas +me +esse +eles +você +essa +num +nem +suas +meu +às +minha +numa +pelos +elas +qual +nós +lhe +deles +essas +esses +pelas +este +dele +tu +te +vocês +vos +lhes +meus +minhas +teu +tua +teus +tuas +nosso +nossa +nossos +nossas +dela +delas +esta +estes +estas +aquele +aquela +aqueles +aquelas +isto +aquilo +estou +está +estamos +estão +estive +esteve +estivemos +estiveram +estava +estávamos +estavam +estivera +estivéramos +esteja +estejamos +estejam +estivesse +estivéssemos +estivessem +estiver +estivermos +estiverem +hei +há +havemos +hão +houve +houvemos +houveram +houvera +houvéramos +haja +hajamos +hajam +houvesse +houvéssemos +houvessem +houver +houvermos +houverem +houverei +houverá +houveremos +houverão +houveria +houveríamos +houveriam +sou +somos +são +era +éramos +eram +fui +foi +fomos +foram +fora +fôramos +seja +sejamos +sejam +fosse +fôssemos +fossem +for +formos +forem +serei +será +seremos +serão +seria +seríamos +seriam +tenho +tem +temos +tém +tinha +tínhamos +tinham +tive +teve +tivemos +tiveram +tivera +tivéramos +tenha +tenhamos +tenham +tivesse +tivéssemos +tivessem +tiver +tivermos +tiverem +terei +terá +teremos +terão +teria +teríamos +teriam \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/russian.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/russian.txt new file mode 100644 index 000000000000..8a800b74497d --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/russian.txt @@ -0,0 +1,151 @@ +и +в +во +не +что +он +на +я +с +со +как +а +то +все +она +так +его +но +да +ты +к +у +же +вы +за +бы +по +только +ее +мне +было +вот +от +меня +еще +нет +о +из +ему +теперь +когда +даже +ну +вдруг +ли +если +уже +или +ни +быть +был +него +до +вас +нибудь +опять +уж +вам +ведь +там +потом +себя +ничего +ей +может +они +тут +где +есть +надо +ней +для +мы +тебя +их +чем +была +сам +чтоб +без +будто +чего +раз +тоже +себе +под +будет +ж +тогда +кто +этот +того +потому +этого +какой +совсем +ним +здесь +этом +один +почти +мой +тем +чтобы +нее +сейчас +были +куда +зачем +всех +никогда +можно +при +наконец +два +об +другой +хоть +после +над +больше +тот +через +эти +нас +про +всего +них +какая +много +разве +три +эту +моя +впрочем +хорошо +свою +этой +перед +иногда +лучше +чуть +том +нельзя +такой +им +более +всегда +конечно +всю +между \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/spanish.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/spanish.txt new file mode 100644 index 000000000000..94f493a8d1e0 --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/spanish.txt @@ -0,0 +1,313 @@ +de +la +que +el +en +y +a +los +del +se +las +por +un +para +con +no +una +su +al +lo +como +más +pero +sus +le +ya +o +este +sí +porque +esta +entre +cuando +muy +sin +sobre +también +me +hasta +hay +donde +quien +desde +todo +nos +durante +todos +uno +les +ni +contra +otros +ese +eso +ante +ellos +e +esto +mí +antes +algunos +qué +unos +yo +otro +otras +otra +él +tanto +esa +estos +mucho +quienes +nada +muchos +cual +poco +ella +estar +estas +algunas +algo +nosotros +mi +mis +tú +te +ti +tu +tus +ellas +nosotras +vosostros +vosostras +os +mío +mía +míos +mías +tuyo +tuya +tuyos +tuyas +suyo +suya +suyos +suyas +nuestro +nuestra +nuestros +nuestras +vuestro +vuestra +vuestros +vuestras +esos +esas +estoy +estás +está +estamos +estáis +están +esté +estés +estemos +estéis +estén +estaré +estarás +estará +estaremos +estaréis +estarán +estaría +estarías +estaríamos +estaríais +estarían +estaba +estabas +estábamos +estabais +estaban +estuve +estuviste +estuvo +estuvimos +estuvisteis +estuvieron +estuviera +estuvieras +estuviéramos +estuvierais +estuvieran +estuviese +estuvieses +estuviésemos +estuvieseis +estuviesen +estando +estado +estada +estados +estadas +estad +he +has +ha +hemos +habéis +han +haya +hayas +hayamos +hayáis +hayan +habré +habrás +habrá +habremos +habréis +habrán +habría +habrías +habríamos +habríais +habrían +había +habías +habíamos +habíais +habían +hube +hubiste +hubo +hubimos +hubisteis +hubieron +hubiera +hubieras +hubiéramos +hubierais +hubieran +hubiese +hubieses +hubiésemos +hubieseis +hubiesen +habiendo +habido +habida +habidos +habidas +soy +eres +es +somos +sois +son +sea +seas +seamos +seáis +sean +seré +serás +será +seremos +seréis +serán +sería +serías +seríamos +seríais +serían +era +eras +éramos +erais +eran +fui +fuiste +fue +fuimos +fuisteis +fueron +fuera +fueras +fuéramos +fuerais +fueran +fuese +fueses +fuésemos +fueseis +fuesen +sintiendo +sentido +sentida +sentidos +sentidas +siente +sentid +tengo +tienes +tiene +tenemos +tenéis +tienen +tenga +tengas +tengamos +tengáis +tengan +tendré +tendrás +tendrá +tendremos +tendréis +tendrán +tendría +tendrías +tendríamos +tendríais +tendrían +tenía +tenías +teníamos +teníais +tenían +tuve +tuviste +tuvo +tuvimos +tuvisteis +tuvieron +tuviera +tuvieras +tuviéramos +tuvierais +tuvieran +tuviese +tuvieses +tuviésemos +tuvieseis +tuviesen +teniendo +tenido +tenida +tenidos +tenidas +tened \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/swedish.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/swedish.txt new file mode 100644 index 000000000000..9fae31c1858a --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/swedish.txt @@ -0,0 +1,114 @@ +och +det +att +i +en +jag +hon +som +han +på +den +med +var +sig +för +så +till +är +men +ett +om +hade +de +av +icke +mig +du +henne +då +sin +nu +har +inte +hans +honom +skulle +hennes +där +min +man +ej +vid +kunde +något +från +ut +när +efter +upp +vi +dem +vara +vad +över +än +dig +kan +sina +här +ha +mot +alla +under +någon +eller +allt +mycket +sedan +ju +denna +själv +detta +åt +utan +varit +hur +ingen +mitt +ni +bli +blev +oss +din +dessa +några +deras +blir +mina +samma +vilken +er +sådan +vår +blivit +dess +inom +mellan +sådant +varför +varje +vilka +ditt +vem +vilket +sitta +sådana +vart +dina +vars +vårt +våra +ert +era +vilkas \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/turkish.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/turkish.txt new file mode 100644 index 000000000000..4e9708d9d2c5 --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/turkish.txt @@ -0,0 +1,53 @@ +acaba +ama +aslında +az +bazı +belki +biri +birkaç +birşey +biz +bu +çok +çünkü +da +daha +de +defa +diye +eğer +en +gibi +hem +hep +hepsi +her +hiç +için +ile +ise +kez +ki +kim +mı +mu +mü +nasıl +ne +neden +nerde +nerede +nereye +niçin +niye +o +sanki +şey +siz +şu +tüm +ve +veya +ya +yani \ No newline at end of file diff --git a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala index 81140d1f7b21..569a5fb99376 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -18,11 +18,11 @@ package org.apache.spark.ml import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala index 7429f9d652ac..6bbe7e1cb213 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala @@ -26,38 +26,39 @@ import com.github.fommil.netlib.BLAS.{getInstance => NativeBLAS} private[ann] object BreezeUtil { // TODO: switch to MLlib BLAS interface - private def transposeString(a: BDM[Double]): String = if (a.isTranspose) "T" else "N" + private def transposeString(A: BDM[Double]): String = if (A.isTranspose) "T" else "N" /** * DGEMM: C := alpha * A * B + beta * C * @param alpha alpha - * @param a A - * @param b B + * @param A A + * @param B B * @param beta beta - * @param c C + * @param C C */ - def dgemm(alpha: Double, a: BDM[Double], b: BDM[Double], beta: Double, c: BDM[Double]): Unit = { + def dgemm(alpha: Double, A: BDM[Double], B: BDM[Double], beta: Double, C: BDM[Double]): Unit = { // TODO: add code if matrices isTranspose!!! - require(a.cols == b.rows, "A & B Dimension mismatch!") - require(a.rows == c.rows, "A & C Dimension mismatch!") - require(b.cols == c.cols, "A & C Dimension mismatch!") - NativeBLAS.dgemm(transposeString(a), transposeString(b), c.rows, c.cols, a.cols, - alpha, a.data, a.offset, a.majorStride, b.data, b.offset, b.majorStride, - beta, c.data, c.offset, c.rows) + require(A.cols == B.rows, "A & B Dimension mismatch!") + require(A.rows == C.rows, "A & C Dimension mismatch!") + require(B.cols == C.cols, "A & C Dimension mismatch!") + NativeBLAS.dgemm(transposeString(A), transposeString(B), C.rows, C.cols, A.cols, + alpha, A.data, A.offset, A.majorStride, B.data, B.offset, B.majorStride, + beta, C.data, C.offset, C.rows) } /** * DGEMV: y := alpha * A * x + beta * y * @param alpha alpha - * @param a A + * @param A A * @param x x * @param beta beta * @param y y */ - def dgemv(alpha: Double, a: BDM[Double], x: BDV[Double], beta: Double, y: BDV[Double]): Unit = { - require(a.cols == x.length, "A & b Dimension mismatch!") - NativeBLAS.dgemv(transposeString(a), a.rows, a.cols, - alpha, a.data, a.offset, a.majorStride, x.data, x.offset, x.stride, + def dgemv(alpha: Double, A: BDM[Double], x: BDV[Double], beta: Double, y: BDV[Double]): Unit = { + require(A.cols == x.length, "A & x Dimension mismatch!") + require(A.rows == y.length, "A & y Dimension mismatch!") + NativeBLAS.dgemv(transposeString(A), A.rows, A.cols, + alpha, A.data, A.offset, A.majorStride, x.data, x.offset, x.stride, beta, y.data, y.offset, y.stride) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala index 3588ac1e95be..0a569c49179a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala @@ -21,7 +21,9 @@ import java.util.Random import breeze.linalg.{*, axpy => Baxpy, DenseMatrix => BDM, DenseVector => BDV, Vector => BV} -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.optimization._ import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom @@ -64,8 +66,9 @@ private[ann] trait Layer extends Serializable { * @return the layer model */ def createModel(initialWeights: BDV[Double]): LayerModel + /** - * Returns the instance of the layer with random generated weights + * Returns the instance of the layer with random generated weights. * * @param weights vector for weights initialization, must be equal to weightSize * @param random random number generator @@ -83,11 +86,11 @@ private[ann] trait LayerModel extends Serializable { val weights: BDV[Double] /** - * Evaluates the data (process the data through the layer) + * Evaluates the data (process the data through the layer). * Output is allocated based on the size provided by the - * LayerModel implementation and the stack (batch) size + * LayerModel implementation and the stack (batch) size. * Developer is responsible for checking the size of output - * when writing to it + * when writing to it. * * @param data data * @param output output (modified in place) @@ -95,11 +98,11 @@ private[ann] trait LayerModel extends Serializable { def eval(data: BDM[Double], output: BDM[Double]): Unit /** - * Computes the delta for back propagation + * Computes the delta for back propagation. * Delta is allocated based on the size provided by the - * LayerModel implementation and the stack (batch) size + * LayerModel implementation and the stack (batch) size. * Developer is responsible for checking the size of - * prevDelta when writing to it + * prevDelta when writing to it. * * @param delta delta of this layer * @param output output of this layer @@ -108,10 +111,10 @@ private[ann] trait LayerModel extends Serializable { def computePrevDelta(delta: BDM[Double], output: BDM[Double], prevDelta: BDM[Double]): Unit /** - * Computes the gradient - * cumGrad is a wrapper on the part of the weight vector - * size of cumGrad is based on weightSize provided by - * implementation of LayerModel + * Computes the gradient. + * cumGrad is a wrapper on the part of the weight vector. + * Size of cumGrad is based on weightSize provided by + * implementation of LayerModel. * * @param delta delta for this layer * @param input input data @@ -197,11 +200,11 @@ private[ann] object AffineLayerModel { } /** - * Initialize weights randomly in the interval - * Uses [Bottou-88] heuristic [-a/sqrt(in); a/sqrt(in)] - * where a is chosen in a such way that the weight variance corresponds + * Initialize weights randomly in the interval. + * Uses [Bottou-88] heuristic [-a/sqrt(in); a/sqrt(in)], + * where `a` is chosen in such a way that the weight variance corresponds * to the points to the maximal curvature of the activation function - * (which is approximately 2.38 for a standard sigmoid) + * (which is approximately 2.38 for a standard sigmoid). * * @param numIn number of inputs * @param numOut number of outputs @@ -306,7 +309,7 @@ private[ann] class FunctionalLayer (val activationFunction: ActivationFunction) /** * Functional layer model. Holds no weights. * - * @param layer functiona layer + * @param layer functional layer */ private[ann] class FunctionalLayerModel private[ann] (val layer: FunctionalLayer) extends LayerModel { @@ -352,6 +355,7 @@ private[ann] trait TopologyModel extends Serializable { * Array of layer models */ val layerModels: Array[LayerModel] + /** * Forward propagation * @@ -410,7 +414,7 @@ private[ml] object FeedForwardTopology { * Creates a multi-layer perceptron * * @param layerSizes sizes of layers including input and output size - * @param softmaxOnTop wether to use SoftMax or Sigmoid function for an output layer. + * @param softmaxOnTop whether to use SoftMax or Sigmoid function for an output layer. * Softmax is default * @return multilayer perceptron topology */ @@ -578,10 +582,10 @@ private[ann] object FeedForwardModel { */ private[ann] class ANNGradient(topology: Topology, dataStacker: DataStacker) extends Gradient { override def compute( - data: Vector, + data: OldVector, label: Double, - weights: Vector, - cumGradient: Vector): Double = { + weights: OldVector, + cumGradient: OldVector): Double = { val (input, target, realBatchSize) = dataStacker.unstack(data) val model = topology.model(weights) model.computeGradient(input, target, cumGradient, realBatchSize) @@ -655,15 +659,15 @@ private[ann] class DataStacker(stackSize: Int, inputSize: Int, outputSize: Int) private[ann] class ANNUpdater extends Updater { override def compute( - weightsOld: Vector, - gradient: Vector, + weightsOld: OldVector, + gradient: OldVector, stepSize: Double, iter: Int, - regParam: Double): (Vector, Double) = { + regParam: Double): (OldVector, Double) = { val thisIterStepSize = stepSize val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector Baxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) - (Vectors.fromBreeze(brzWeights), 0) + (OldVectors.fromBreeze(brzWeights), 0) } } @@ -806,7 +810,9 @@ private[ml] class FeedForwardTrainer( getWeights } // TODO: deprecate standard optimizer because it needs Vector - val newWeights = optimizer.optimize(dataStacker.stack(data), w) + val newWeights = optimizer.optimize(dataStacker.stack(data).map { v => + (v._1, OldVectors.fromML(v._2)) + }, w) topology.model(newWeights) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala index 2c29eeb01a92..12b9732a4c3d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.attribute import scala.collection.mutable.ArrayBuffer import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.VectorUDT +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types.{Metadata, MetadataBuilder, StructField} /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index bc5fe35ad4a5..e35b04a1cf42 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -20,10 +20,10 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkException import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.shared.HasRawPredictionCol import org.apache.spark.ml.util.{MetadataUtils, SchemaUtils} -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index 31a69d49a078..881dcefb79be 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -22,13 +22,13 @@ import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.DecisionTreeModelReadWrite._ import org.apache.spark.ml.tree.impl.RandomForest import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index acc04582b869..f843df449c61 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -24,14 +24,14 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{PredictionModel, Predictor} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.GradientBoostedTrees import org.apache.spark.ml.util._ import org.apache.spark.ml.util.DefaultParamsReader.Metadata -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index d2d4e249b420..0ab4459bdb9d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -27,12 +27,13 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.linalg.BLAS._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics -import org.apache.spark.mllib.linalg._ -import org.apache.spark.mllib.linalg.BLAS._ +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD @@ -366,9 +367,10 @@ class LogisticRegression @Since("1.2.0") ( Vectors.zeros(if ($(fitIntercept)) numFeatures + 1 else numFeatures) if (optInitialModel.isDefined && optInitialModel.get.coefficients.size != numFeatures) { - val vec = optInitialModel.get.coefficients + val vecSize = optInitialModel.get.coefficients.size logWarning( - s"Initial coefficients provided $vec did not match the expected size $numFeatures") + s"Initial coefficients will be ignored!! As its size $vecSize did not match the " + + s"expected size $numFeatures") } if (optInitialModel.isDefined && optInitialModel.get.coefficients.size == numFeatures) { @@ -612,7 +614,7 @@ class LogisticRegressionModel private[spark] ( } /** - * Returns a [[MLWriter]] instance for this ML instance. + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. * * For [[LogisticRegressionModel]], this does NOT currently save the training [[summary]]. * An option to save [[summary]] may be added in the future. @@ -744,7 +746,7 @@ private[classification] class MultiClassSummarizer extends Serializable { def countInvalid: Long = totalInvalidCnt /** @return The number of distinct labels in the input dataset. */ - def numClasses: Int = distinctMap.keySet.max + 1 + def numClasses: Int = if (distinctMap.isEmpty) 0 else distinctMap.keySet.max + 1 /** @return The weightSum of each label in the input dataset. */ def histogram: Array[Double] = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index 72cf55f6bb99..c4e882240ffd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -24,30 +24,30 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.ml.ann.{FeedForwardTopology, FeedForwardTrainer} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasMaxIter, HasSeed, HasStepSize, HasTol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.sql.Dataset /** Params for Multilayer Perceptron. */ -private[ml] trait MultilayerPerceptronParams extends PredictorParams +private[classification] trait MultilayerPerceptronParams extends PredictorParams with HasSeed with HasMaxIter with HasTol with HasStepSize { /** * Layer sizes including input size and output size. - * Default: Array(1, 1) * * @group param */ + @Since("1.5.0") final val layers: IntArrayParam = new IntArrayParam(this, "layers", - "Sizes of layers from input layer to output layer" + - " E.g., Array(780, 100, 10) means 780 inputs, " + + "Sizes of layers from input layer to output layer. " + + "E.g., Array(780, 100, 10) means 780 inputs, " + "one hidden layer with 100 neurons and output layer of 10 neurons.", - (t: Array[Int]) => t.forall(ParamValidators.gt(0)) && t.length > 1 - ) + (t: Array[Int]) => t.forall(ParamValidators.gt(0)) && t.length > 1) /** @group getParam */ + @Since("1.5.0") final def getLayers: Array[Int] = $(layers) /** @@ -59,42 +59,49 @@ private[ml] trait MultilayerPerceptronParams extends PredictorParams * * @group expertParam */ + @Since("1.5.0") final val blockSize: IntParam = new IntParam(this, "blockSize", "Block size for stacking input data in matrices. Data is stacked within partitions." + " If block size is more than remaining data in a partition then " + "it is adjusted to the size of this data. Recommended size is between 10 and 1000", ParamValidators.gt(0)) - /** @group getParam */ + /** @group expertGetParam */ + @Since("1.5.0") final def getBlockSize: Int = $(blockSize) /** - * Allows setting the solver: minibatch gradient descent (gd) or l-bfgs. - * l-bfgs is the default one. + * The solver algorithm for optimization. + * Supported options: "gd" (minibatch gradient descent) or "l-bfgs". + * Default: "l-bfgs" * * @group expertParam */ + @Since("2.0.0") final val solver: Param[String] = new Param[String](this, "solver", - " Allows setting the solver: minibatch gradient descent (gd) or l-bfgs. " + - " l-bfgs is the default one.", - ParamValidators.inArray[String](Array("gd", "l-bfgs"))) + "The solver algorithm for optimization. Supported options: " + + s"${MultilayerPerceptronClassifier.supportedSolvers.mkString(", ")}. (Default l-bfgs)", + ParamValidators.inArray[String](MultilayerPerceptronClassifier.supportedSolvers)) - /** @group getParam */ - final def getOptimizer: String = $(solver) + /** @group expertGetParam */ + @Since("2.0.0") + final def getSolver: String = $(solver) /** - * Model weights. Can be returned either after training or after explicit setting + * The initial weights of the model. * * @group expertParam */ - final val weights: Param[Vector] = new Param[Vector](this, "weights", - " Sets the weights of the model ") - - /** @group getParam */ - final def getWeights: Vector = $(weights) + @Since("2.0.0") + final val initialWeights: Param[Vector] = new Param[Vector](this, "initialWeights", + "The initial weights of the model") + /** @group expertGetParam */ + @Since("2.0.0") + final def getInitialWeights: Vector = $(initialWeights) - setDefault(maxIter -> 100, tol -> 1e-4, blockSize -> 128, solver -> "l-bfgs", stepSize -> 0.03) + setDefault(maxIter -> 100, tol -> 1e-4, blockSize -> 128, + solver -> MultilayerPerceptronClassifier.LBFGS, stepSize -> 0.03) } /** Label to vector converter. */ @@ -145,14 +152,32 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( @Since("1.5.0") def this() = this(Identifiable.randomUID("mlpc")) - /** @group setParam */ + /** + * Sets the value of param [[layers]]. + * + * @group setParam + */ @Since("1.5.0") def setLayers(value: Array[Int]): this.type = set(layers, value) - /** @group setParam */ + /** + * Sets the value of param [[blockSize]]. + * Default is 128. + * + * @group expertSetParam + */ @Since("1.5.0") def setBlockSize(value: Int): this.type = set(blockSize, value) + /** + * Sets the value of param [[solver]]. + * Default is "l-bfgs". + * + * @group expertSetParam + */ + @Since("2.0.0") + def setSolver(value: String): this.type = set(solver, value) + /** * Set the maximum number of iterations. * Default is 100. @@ -181,12 +206,21 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( def setSeed(value: Long): this.type = set(seed, value) /** - * Sets the model weights. + * Sets the value of param [[initialWeights]]. * - * @group expertParam + * @group expertSetParam + */ + @Since("2.0.0") + def setInitialWeights(value: Vector): this.type = set(initialWeights, value) + + /** + * Sets the value of param [[stepSize]] (applicable only for solver "gd"). + * Default is 0.03. + * + * @group setParam */ @Since("2.0.0") - def setWeights(value: Vector): this.type = set(weights, value) + def setStepSize(value: Double): this.type = set(stepSize, value) @Since("1.5.0") override def copy(extra: ParamMap): MultilayerPerceptronClassifier = defaultCopy(extra) @@ -204,16 +238,26 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( val labels = myLayers.last val lpData = extractLabeledPoints(dataset) val data = lpData.map(lp => LabelConverter.encodeLabeledPoint(lp, labels)) - val topology = FeedForwardTopology.multiLayerPerceptron(myLayers, true) + val topology = FeedForwardTopology.multiLayerPerceptron(myLayers, softmaxOnTop = true) val trainer = new FeedForwardTrainer(topology, myLayers(0), myLayers.last) - if (isDefined(weights)) { - trainer.setWeights($(weights)) + if (isDefined(initialWeights)) { + trainer.setWeights($(initialWeights)) } else { trainer.setSeed($(seed)) } - trainer.LBFGSOptimizer - .setConvergenceTol($(tol)) - .setNumIterations($(maxIter)) + if ($(solver) == MultilayerPerceptronClassifier.LBFGS) { + trainer.LBFGSOptimizer + .setConvergenceTol($(tol)) + .setNumIterations($(maxIter)) + } else if ($(solver) == MultilayerPerceptronClassifier.GD) { + trainer.SGDOptimizer + .setNumIterations($(maxIter)) + .setConvergenceTol($(tol)) + .setStepSize($(stepSize)) + } else { + throw new IllegalArgumentException( + s"The solver $solver is not supported by MultilayerPerceptronClassifier.") + } trainer.setStackSize($(blockSize)) val mlpModel = trainer.train(data) new MultilayerPerceptronClassificationModel(uid, myLayers, mlpModel.weights) @@ -224,6 +268,15 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( object MultilayerPerceptronClassifier extends DefaultParamsReadable[MultilayerPerceptronClassifier] { + /** String name for "l-bfgs" solver. */ + private[classification] val LBFGS = "l-bfgs" + + /** String name for "gd" (minibatch gradient descent) solver. */ + private[classification] val GD = "gd" + + /** Set of solvers that MultilayerPerceptronClassifier supports. */ + private[classification] val supportedSolvers = Array(LBFGS, GD) + @Since("2.0.0") override def load(path: String): MultilayerPerceptronClassifier = super.load(path) } @@ -250,7 +303,9 @@ class MultilayerPerceptronClassificationModel private[ml] ( @Since("1.6.0") override val numFeatures: Int = layers.head - private val mlpModel = FeedForwardTopology.multiLayerPerceptron(layers, true).model(weights) + private val mlpModel = FeedForwardTopology + .multiLayerPerceptron(layers, softmaxOnTop = true) + .model(weights) /** * Returns layers in a Java List. diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index 267d63b51eb6..a98bdeca6b72 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -22,14 +22,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.PredictorParams +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} import org.apache.spark.ml.util._ import org.apache.spark.mllib.classification.{NaiveBayes => OldNaiveBayes} import org.apache.spark.mllib.classification.{NaiveBayesModel => OldNaiveBayesModel} -import org.apache.spark.mllib.linalg._ -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.Dataset /** * Params for Naive Bayes Classifiers. @@ -102,7 +102,8 @@ class NaiveBayes @Since("1.5.0") ( setDefault(modelType -> OldNaiveBayes.Multinomial) override protected def train(dataset: Dataset[_]): NaiveBayesModel = { - val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) + val oldDataset: RDD[OldLabeledPoint] = + extractLabeledPoints(dataset).map(OldLabeledPoint.fromML) val oldModel = OldNaiveBayes.train(oldDataset, $(smoothing), $(modelType)) NaiveBayesModel.fromOld(oldModel, this) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index f10c60a78df1..047a378b79aa 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -32,9 +32,9 @@ import org.apache.spark.SparkContext import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml._ import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.{Param, ParamMap, ParamPair, Params} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index d00fee12b08c..59277d0f42b3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -18,9 +18,9 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.linalg.{DenseVector, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils -import org.apache.spark.mllib.linalg.{DenseVector, Vector, Vectors, VectorUDT} import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, StructType} diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index 1d33ae83c211..b3c074f83925 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -21,13 +21,13 @@ import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.RandomForest import org.apache.spark.ml.util._ import org.apache.spark.ml.util.DefaultParamsReader.Metadata -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index 6cc9117da3fe..afb1080b9b7d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -21,12 +21,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.mllib.clustering. - {BisectingKMeans => MLlibBisectingKMeans, BisectingKMeansModel => MLlibBisectingKMeansModel} -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.clustering.{BisectingKMeans => MLlibBisectingKMeans, BisectingKMeansModel => MLlibBisectingKMeansModel} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{IntegerType, StructType} @@ -39,23 +41,27 @@ private[clustering] trait BisectingKMeansParams extends Params with HasMaxIter with HasFeaturesCol with HasSeed with HasPredictionCol { /** - * Set the number of clusters to create (k). Must be > 1. Default: 2. + * The desired number of leaf clusters. Must be > 1. Default: 4. + * The actual number could be smaller if there are no divisible leaf clusters. * @group param */ @Since("2.0.0") - final val k = new IntParam(this, "k", "number of clusters to create", (x: Int) => x > 1) + final val k = new IntParam(this, "k", "The desired number of leaf clusters. " + + "Must be > 1.", ParamValidators.gt(1)) /** @group getParam */ @Since("2.0.0") def getK: Int = $(k) - /** @group expertParam */ + /** + * The minimum number of points (if >= 1.0) or the minimum proportion + * of points (if < 1.0) of a divisible cluster (default: 1.0). + * @group expertParam + */ @Since("2.0.0") - final val minDivisibleClusterSize = new DoubleParam( - this, - "minDivisibleClusterSize", - "the minimum number of points (if >= 1.0) or the minimum proportion", - (value: Double) => value > 0) + final val minDivisibleClusterSize = new DoubleParam(this, "minDivisibleClusterSize", + "The minimum number of points (if >= 1.0) or the minimum proportion " + + "of points (if < 1.0) of a divisible cluster.", ParamValidators.gt(0.0)) /** @group expertGetParam */ @Since("2.0.0") @@ -76,7 +82,7 @@ private[clustering] trait BisectingKMeansParams extends Params * :: Experimental :: * Model fitted by BisectingKMeans. * - * @param parentModel a model trained by spark.mllib.clustering.BisectingKMeans. + * @param parentModel a model trained by [[org.apache.spark.mllib.clustering.BisectingKMeans]]. */ @Since("2.0.0") @Experimental @@ -105,7 +111,7 @@ class BisectingKMeansModel private[ml] ( private[clustering] def predict(features: Vector): Int = parentModel.predict(features) @Since("2.0.0") - def clusterCenters: Array[Vector] = parentModel.clusterCenters + def clusterCenters: Array[Vector] = parentModel.clusterCenters.map(_.asML) /** * Computes the sum of squared distances between the input points and their corresponding cluster @@ -115,7 +121,7 @@ class BisectingKMeansModel private[ml] ( def computeCost(dataset: Dataset[_]): Double = { SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new VectorUDT) val data = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => point } - parentModel.computeCost(data) + parentModel.computeCost(data.map(OldVectors.fromML)) } @Since("2.0.0") @@ -216,7 +222,9 @@ class BisectingKMeans @Since("2.0.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): BisectingKMeansModel = { - val rdd = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => point } + val rdd: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(point: Vector) => OldVectors.fromML(point) + } val bkm = new MLlibBisectingKMeans() .setK($(k)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index ac86e4ce25e8..88b6b27e62d9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -25,13 +25,14 @@ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.impl.Utils.EPSILON import org.apache.spark.ml.linalg._ -import org.apache.spark.ml.param.{IntParam, ParamMap, Params} +import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.stat.distribution.MultivariateGaussian import org.apache.spark.ml.util._ import org.apache.spark.mllib.clustering.{GaussianMixture => MLlibGM} import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Matrix => OldMatrix, Vector => OldVector, Vectors => OldVectors, VectorUDT => OldVectorUDT} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{IntegerType, StructType} @@ -44,11 +45,12 @@ private[clustering] trait GaussianMixtureParams extends Params with HasMaxIter w with HasSeed with HasPredictionCol with HasProbabilityCol with HasTol { /** - * Set the number of clusters to create (k). Must be > 1. Default: 2. + * Number of independent Gaussians in the mixture model. Must be > 1. Default: 2. * @group param */ @Since("2.0.0") - final val k = new IntParam(this, "k", "number of clusters to create", (x: Int) => x > 1) + final val k = new IntParam(this, "k", "Number of independent Gaussians in the mixture model. " + + "Must be > 1.", ParamValidators.gt(1)) /** @group getParam */ @Since("2.0.0") @@ -94,8 +96,8 @@ class GaussianMixtureModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - val predUDF = udf((vector: OldVector) => predict(vector.asML)) - val probUDF = udf((vector: OldVector) => OldVectors.fromML(predictProbability(vector.asML))) + val predUDF = udf((vector: Vector) => predict(vector)) + val probUDF = udf((vector: Vector) => predictProbability(vector)) dataset.withColumn($(predictionCol), predUDF(col($(featuresCol)))) .withColumn($(probabilityCol), probUDF(col($(featuresCol)))) } @@ -137,6 +139,13 @@ class GaussianMixtureModel private[ml] ( sqlContext.createDataFrame(modelGaussians).toDF("mean", "cov") } + /** + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. + * + * For [[GaussianMixtureModel]], this does NOT currently save the training [[summary]]. + * An option to save [[summary]] may be added in the future. + * + */ @Since("2.0.0") override def write: MLWriter = new GaussianMixtureModel.GaussianMixtureModelWriter(this) @@ -248,6 +257,21 @@ object GaussianMixtureModel extends MLReadable[GaussianMixtureModel] { /** * :: Experimental :: * Gaussian Mixture clustering. + * + * This class performs expectation maximization for multivariate Gaussian + * Mixture Models (GMMs). A GMM represents a composite distribution of + * independent Gaussian distributions with associated "mixing" weights + * specifying each's contribution to the composite. + * + * Given a set of sample points, this class will maximize the log-likelihood + * for a mixture of k Gaussians, iterating until the log-likelihood changes by + * less than convergenceTol, or until it has reached the max number of iterations. + * While this process is generally guaranteed to converge, it is not guaranteed + * to find a global optimum. + * + * Note: For high-dimensional data (with many features), this algorithm may perform poorly. + * This is due to high-dimensional data (a) making it difficult to cluster at all (based + * on statistical/theoretical arguments) and (b) numerical issues with Gaussian distributions. */ @Since("2.0.0") @Experimental @@ -296,7 +320,9 @@ class GaussianMixture @Since("2.0.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): GaussianMixtureModel = { - val rdd = dataset.select(col($(featuresCol))).rdd.map { case Row(point: OldVector) => point } + val rdd: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(point: Vector) => OldVectors.fromML(point) + } val algo = new MLlibGM() .setK($(k)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 7c9ac02521ff..790ef1fe8dc9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -22,11 +22,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} -import org.apache.spark.ml.param.{IntParam, Param, ParamMap, Params} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} +import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.clustering.{KMeans => MLlibKMeans, KMeansModel => MLlibKMeansModel} -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{IntegerType, StructType} @@ -38,11 +41,12 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe with HasSeed with HasPredictionCol with HasTol { /** - * Set the number of clusters to create (k). Must be > 1. Default: 2. + * The number of clusters to create (k). Must be > 1. Default: 2. * @group param */ @Since("1.5.0") - final val k = new IntParam(this, "k", "number of clusters to create", (x: Int) => x > 1) + final val k = new IntParam(this, "k", "The number of clusters to create. " + + "Must be > 1.", ParamValidators.gt(1)) /** @group getParam */ @Since("1.5.0") @@ -55,7 +59,8 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe * @group expertParam */ @Since("1.5.0") - final val initMode = new Param[String](this, "initMode", "initialization algorithm", + final val initMode = new Param[String](this, "initMode", "The initialization algorithm. " + + "Supported options: 'random' and 'k-means||'.", (value: String) => MLlibKMeans.validateInitMode(value)) /** @group expertGetParam */ @@ -68,8 +73,8 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe * @group expertParam */ @Since("1.5.0") - final val initSteps = new IntParam(this, "initSteps", "number of steps for k-means||", - (value: Int) => value > 0) + final val initSteps = new IntParam(this, "initSteps", "The number of steps for k-means|| " + + "initialization mode. Must be > 0.", ParamValidators.gt(0)) /** @group expertGetParam */ @Since("1.5.0") @@ -105,6 +110,14 @@ class KMeansModel private[ml] ( copyValues(copied, extra) } + /** @group setParam */ + @Since("2.0.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + /** @group setParam */ + @Since("2.0.0") + def setPredictionCol(value: String): this.type = set(predictionCol, value) + @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { val predictUDF = udf((vector: Vector) => predict(vector)) @@ -119,7 +132,7 @@ class KMeansModel private[ml] ( private[clustering] def predict(features: Vector): Int = parentModel.predict(features) @Since("1.5.0") - def clusterCenters: Array[Vector] = parentModel.clusterCenters + def clusterCenters: Array[Vector] = parentModel.clusterCenters.map(_.asML) /** * Return the K-means cost (sum of squared distances of points to their nearest center) for this @@ -129,10 +142,19 @@ class KMeansModel private[ml] ( @Since("2.0.0") def computeCost(dataset: Dataset[_]): Double = { SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new VectorUDT) - val data = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => point } + val data: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(point: Vector) => OldVectors.fromML(point) + } parentModel.computeCost(data) } + /** + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. + * + * For [[KMeansModel]], this does NOT currently save the training [[summary]]. + * An option to save [[summary]] may be added in the future. + * + */ @Since("1.6.0") override def write: MLWriter = new KMeansModel.KMeansModelWriter(this) @@ -172,6 +194,12 @@ object KMeansModel extends MLReadable[KMeansModel] { /** Helper class for storing model data */ private case class Data(clusterIdx: Int, clusterCenter: Vector) + /** + * We store all cluster centers in a single row and use this class to store model data by + * Spark 1.6 and earlier. A model can be loaded from such older data for backward compatibility. + */ + private case class OldData(clusterCenters: Array[OldVector]) + /** [[MLWriter]] instance for [[KMeansModel]] */ private[KMeansModel] class KMeansModelWriter(instance: KMeansModel) extends MLWriter { @@ -198,12 +226,19 @@ object KMeansModel extends MLReadable[KMeansModel] { import sqlContext.implicits._ val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - val dataPath = new Path(path, "data").toString - val data: Dataset[Data] = sqlContext.read.parquet(dataPath).as[Data] - val clusterCenters = data.collect().sortBy(_.clusterIdx).map(_.clusterCenter) - val model = new KMeansModel(metadata.uid, new MLlibKMeansModel(clusterCenters)) + val versionRegex = "([0-9]+)\\.(.+)".r + val versionRegex(major, _) = metadata.sparkVersion + + val clusterCenters = if (major.toInt >= 2) { + val data: Dataset[Data] = sqlContext.read.parquet(dataPath).as[Data] + data.collect().sortBy(_.clusterIdx).map(_.clusterCenter).map(OldVectors.fromML) + } else { + // Loads KMeansModel stored with the old format used by Spark 1.6 and earlier. + sqlContext.read.parquet(dataPath).as[OldData].head().clusterCenters + } + val model = new KMeansModel(metadata.uid, new MLlibKMeansModel(clusterCenters)) DefaultParamsReader.getAndSetParams(model, metadata) model } @@ -269,7 +304,9 @@ class KMeans @Since("1.5.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): KMeansModel = { - val rdd = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => point } + val rdd: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(point: Vector) => OldVectors.fromML(point) + } val instr = Instrumentation.create(this, rdd) instr.logParams(featuresCol, predictionCol, k, initMode, initSteps, maxIter, seed, tol) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 38ecc5a102c1..ec60991af64f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Matrix, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasCheckpointInterval, HasFeaturesCol, HasMaxIter, HasSeed} import org.apache.spark.ml.util._ @@ -30,7 +31,10 @@ import org.apache.spark.mllib.clustering.{DistributedLDAModel => OldDistributedL LDAOptimizer => OldLDAOptimizer, LocalLDAModel => OldLocalLDAModel, OnlineLDAOptimizer => OldOnlineLDAOptimizer} import org.apache.spark.mllib.impl.PeriodicCheckpointer -import org.apache.spark.mllib.linalg.{Matrix, Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Vector => OldVector, + Vectors => OldVectors} +import org.apache.spark.mllib.linalg.MatrixImplicits._ +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.functions.{col, monotonicallyIncreasingId, udf} @@ -46,8 +50,8 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM * @group param */ @Since("1.6.0") - final val k = new IntParam(this, "k", "number of topics (clusters) to infer", - ParamValidators.gt(1)) + final val k = new IntParam(this, "k", "The number of topics (clusters) to infer. " + + "Must be > 1.", ParamValidators.gt(1)) /** @group getParam */ @Since("1.6.0") @@ -161,7 +165,7 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM */ @Since("1.6.0") final val optimizer = new Param[String](this, "optimizer", "Optimizer or inference" + - " algorithm used to estimate the LDA model. Supported: " + supportedOptimizers.mkString(", "), + " algorithm used to estimate the LDA model. Supported: " + supportedOptimizers.mkString(", "), (o: String) => ParamValidators.inArray(supportedOptimizers).apply(o.toLowerCase)) /** @group getParam */ @@ -405,7 +409,11 @@ sealed abstract class LDAModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { if ($(topicDistributionCol).nonEmpty) { - val t = udf(oldLocalModel.getTopicDistributionMethod(sparkSession.sparkContext)) + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer = oldLocalModel.getTopicDistributionMethod(sparkSession.sparkContext) + + val t = udf { (v: Vector) => transformer(OldVectors.fromML(v)).asML } dataset.withColumn($(topicDistributionCol), t(col($(featuresCol)))).toDF } else { logWarning("LDAModel.transform was called without any output columns. Set an output column" + @@ -437,7 +445,7 @@ sealed abstract class LDAModel private[ml] ( * collecting a large amount of data to the driver (on the order of vocabSize x k). */ @Since("1.6.0") - def topicsMatrix: Matrix = oldLocalModel.topicsMatrix + def topicsMatrix: Matrix = oldLocalModel.topicsMatrix.asML /** Indicates whether this instance is of type [[DistributedLDAModel]] */ @Since("1.6.0") @@ -872,13 +880,13 @@ class LDA @Since("1.6.0") ( private[clustering] object LDA extends DefaultParamsReadable[LDA] { /** Get dataset for spark.mllib LDA */ - def getOldDataset(dataset: Dataset[_], featuresCol: String): RDD[(Long, Vector)] = { + def getOldDataset(dataset: Dataset[_], featuresCol: String): RDD[(Long, OldVector)] = { dataset .withColumn("docId", monotonicallyIncreasingId()) .select("docId", featuresCol) .rdd .map { case Row(docId: Long, features: Vector) => - (docId, features) + (docId, OldVectors.fromML(features)) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index 0cbc391d96f8..bff72b20e1c3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -18,11 +18,11 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.{Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index 3d89843a0b71..390e9b6444c7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.types.DoubleType /** * :: Experimental :: - * Evaluator for multiclass classification, which expects two input columns: score and label. + * Evaluator for multiclass classification, which expects two input columns: prediction and label. */ @Since("1.5.0") @Experimental @@ -40,15 +40,15 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid /** * param for metric name in evaluation (supports `"f1"` (default), `"precision"`, `"recall"`, - * `"weightedPrecision"`, `"weightedRecall"`) + * `"weightedPrecision"`, `"weightedRecall"`, `"accuracy"`) * @group param */ @Since("1.5.0") val metricName: Param[String] = { val allowedParams = ParamValidators.inArray(Array("f1", "precision", - "recall", "weightedPrecision", "weightedRecall")) + "recall", "weightedPrecision", "weightedRecall", "accuracy")) new Param(this, "metricName", "metric name in evaluation " + - "(f1|precision|recall|weightedPrecision|weightedRecall)", allowedParams) + "(f1|precision|recall|weightedPrecision|weightedRecall|accuracy)", allowedParams) } /** @group getParam */ @@ -86,18 +86,13 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid case "recall" => metrics.recall case "weightedPrecision" => metrics.weightedPrecision case "weightedRecall" => metrics.weightedRecall + case "accuracy" => metrics.accuracy } metric } @Since("1.5.0") - override def isLargerBetter: Boolean = $(metricName) match { - case "f1" => true - case "precision" => true - case "recall" => true - case "weightedPrecision" => true - case "weightedRecall" => true - } + override def isLargerBetter: Boolean = true @Since("1.5.0") override def copy(extra: ParamMap): MulticlassClassificationEvaluator = defaultCopy(extra) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index 898ac2cc8941..318c8b8b2f7d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -22,10 +22,10 @@ import scala.collection.mutable.ArrayBuilder import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.BinaryAttribute +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg._ import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -104,9 +104,9 @@ final class Binarizer(override val uid: String) case DoubleType => BinaryAttribute.defaultAttr.withName(outputColName).toStructField() case _: VectorUDT => - new StructField(outputColName, new VectorUDT, true) - case other => - throw new IllegalArgumentException(s"Data type $other is not supported.") + StructField(outputColName, new VectorUDT) + case _ => + throw new IllegalArgumentException(s"Data type $inputType is not supported.") } if (schema.fieldNames.contains(outputColName)) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index cfecae7e0b15..e73a8f5d6608 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -22,12 +22,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml._ import org.apache.spark.ml.attribute.{AttributeGroup, _} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DoubleType, StructField, StructType} @@ -80,17 +82,18 @@ final class ChiSqSelector(override val uid: String) @Since("2.0.0") override def fit(dataset: Dataset[_]): ChiSqSelectorModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(labelCol), $(featuresCol)).rdd.map { - case Row(label: Double, features: Vector) => - LabeledPoint(label, features) - } + val input: RDD[OldLabeledPoint] = + dataset.select(col($(labelCol)).cast(DoubleType), col($(featuresCol))).rdd.map { + case Row(label: Double, features: Vector) => + OldLabeledPoint(label, OldVectors.fromML(features)) + } val chiSqSelector = new feature.ChiSqSelector($(numTopFeatures)).fit(input) copyValues(new ChiSqSelectorModel(uid, chiSqSelector).setParent(this)) } override def transformSchema(schema: StructType): StructType = { SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) - SchemaUtils.checkColumnType(schema, $(labelCol), DoubleType) + SchemaUtils.checkNumericType(schema, $(labelCol)) SchemaUtils.appendColumn(schema, $(outputCol), new VectorUDT) } @@ -132,7 +135,11 @@ final class ChiSqSelectorModel private[ml] ( override def transform(dataset: Dataset[_]): DataFrame = { val transformedSchema = transformSchema(dataset.schema, logging = true) val newField = transformedSchema.last - val selector = udf { chiSqSelector.transform _ } + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer: Vector => Vector = v => chiSqSelector.transform(OldVectors.fromML(v)).asML + + val selector = udf(transformer) dataset.withColumn($(outputCol), selector(col($(featuresCol))), newField.metadata) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala index 3fbfce9d48dd..fc4885bf4ba8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala @@ -21,10 +21,10 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.broadcast.Broadcast import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vectors, VectorUDT} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala index a6f878151de7..301358ef1226 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala @@ -21,9 +21,9 @@ import edu.emory.mathcs.jtransforms.dct._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.BooleanParam import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.sql.types.DataType /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala index 1b0a9a12e83b..91989c3d2f5d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -19,10 +19,12 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.Param import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable} import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.sql.types.DataType /** @@ -52,7 +54,7 @@ class ElementwiseProduct(override val uid: String) override protected def createTransformFunc: Vector => Vector = { require(params.contains(scalingVec), s"transformation requires a weight vector") val elemScaler = new feature.ElementwiseProduct($(scalingVec)) - elemScaler.transform + v => elemScaler.transform(v) } override protected def outputDataType: DataType = new VectorUDT() diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala index 66ae91cfc097..94e1825ba61e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala @@ -86,7 +86,8 @@ class HashingTF(override val uid: String) override def transform(dataset: Dataset[_]): DataFrame = { val outputSchema = transformSchema(dataset.schema) val hashingTF = new feature.HashingTF($(numFeatures)).setBinary($(binary)) - val t = udf { terms: Seq[_] => hashingTF.transform(terms) } + // TODO: Make the hashingTF.transform natively in ml framework to avoid extra conversion. + val t = udf { terms: Seq[_] => hashingTF.transform(terms).asML } val metadata = outputSchema($(outputCol)).metadata dataset.select(col("*"), t(col($(inputCol))).as($(outputCol), metadata)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala index 5075b78c9856..08beda6d7515 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -21,11 +21,13 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml._ +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType @@ -36,12 +38,12 @@ import org.apache.spark.sql.types.StructType private[feature] trait IDFBase extends Params with HasInputCol with HasOutputCol { /** - * The minimum of documents in which a term should appear. + * The minimum number of documents in which a term should appear. * Default: 0 * @group param */ final val minDocFreq = new IntParam( - this, "minDocFreq", "minimum of documents in which a term should appear for filtering") + this, "minDocFreq", "minimum number of documents in which a term should appear for filtering") setDefault(minDocFreq -> 0) @@ -79,7 +81,9 @@ final class IDF(override val uid: String) extends Estimator[IDFModel] with IDFBa @Since("2.0.0") override def fit(dataset: Dataset[_]): IDFModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v } + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val idf = new feature.IDF($(minDocFreq)).fit(input) copyValues(new IDFModel(uid, idf).setParent(this)) } @@ -119,7 +123,8 @@ class IDFModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val idf = udf { vec: Vector => idfModel.transform(vec) } + // TODO: Make the idfModel.transform natively in ml framework to avoid extra conversion. + val idf = udf { vec: Vector => idfModel.transform(OldVectors.fromML(vec)).asML } dataset.withColumn($(outputCol), idf(col($(inputCol)))) } @@ -134,7 +139,7 @@ class IDFModel private[ml] ( /** Returns the IDF vector. */ @Since("1.6.0") - def idf: Vector = idfModel.idf + def idf: Vector = idfModel.idf.asML @Since("1.6.0") override def write: MLWriter = new IDFModelWriter(this) @@ -166,7 +171,7 @@ object IDFModel extends MLReadable[IDFModel] { .select("idf") .head() val idf = data.getAs[Vector](0) - val model = new IDFModel(metadata.uid, new feature.IDFModel(idf)) + val model = new IDFModel(metadata.uid, new feature.IDFModel(OldVectors.fromML(idf))) DefaultParamsReader.getAndSetParams(model, metadata) model } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala index 12176757aee3..cce3ca45ccd8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector /** * Class that represents an instance of weighted data point with label and features. diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala index 9ca34e9ae22f..fa65ff987917 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala @@ -26,7 +26,7 @@ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.ml.Transformer -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala new file mode 100644 index 000000000000..f7f1d4203959 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala @@ -0,0 +1,38 @@ +/* + * 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.ml.feature + +import scala.beans.BeanInfo + +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.linalg.Vector + +/** + * Class that represents the features and labels of a data point. + * + * @param label Label for this data point. + * @param features List of features for this data point. + */ +@Since("2.0.0") +@Experimental +@BeanInfo +case class LabeledPoint(@Since("2.0.0") label: Double, @Since("2.0.0") features: Vector) { + override def toString: String = { + s"($label,$features)" + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala index e9df600c8a99..0dffba93ac57 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala @@ -21,11 +21,13 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.{ParamMap, Params} import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -69,7 +71,9 @@ class MaxAbsScaler @Since("2.0.0") (override val uid: String) @Since("2.0.0") override def fit(dataset: Dataset[_]): MaxAbsScalerModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v } + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val summary = Statistics.colStats(input) val minVals = summary.min.toArray val maxVals = summary.max.toArray diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index 125becbb8a5b..c6ff639f2962 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -21,11 +21,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.{DoubleParam, ParamMap, Params} import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -106,7 +109,9 @@ class MinMaxScaler(override val uid: String) @Since("2.0.0") override def fit(dataset: Dataset[_]): MinMaxScalerModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v } + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val summary = Statistics.colStats(input) copyValues(new MinMaxScalerModel(uid, summary.min, summary.max).setParent(this)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala index a603b3f83320..942ac7ebdb3b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala @@ -19,10 +19,11 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.{DoubleParam, ParamValidators} import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} import org.apache.spark.sql.types.DataType /** @@ -52,7 +53,7 @@ class Normalizer(override val uid: String) override protected def createTransformFunc: Vector => Vector = { val normalizer = new feature.Normalizer($(p)) - normalizer.transform + vector => normalizer.transform(OldVectors.fromML(vector)).asML } override protected def outputDataType: DataType = new VectorUDT() diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala index 99357793dbae..3d1e6dd81882 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala @@ -20,10 +20,10 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{DoubleType, NumericType, StructType} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index 9cf722e12169..dbbaa5aa46f4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -21,11 +21,16 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml._ +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.linalg.{DenseMatrix => OldDenseMatrix, DenseVector => OldDenseVector, + Matrices => OldMatrices, Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.MatrixImplicits._ +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -48,7 +53,8 @@ private[feature] trait PCAParams extends Params with HasInputCol with HasOutputC /** * :: Experimental :: - * PCA trains a model to project vectors to a low-dimensional space using PCA. + * PCA trains a model to project vectors to a lower dimensional space of the top [[PCA!.k]] + * principal components. */ @Experimental class PCA (override val uid: String) extends Estimator[PCAModel] with PCAParams @@ -71,7 +77,9 @@ class PCA (override val uid: String) extends Estimator[PCAModel] with PCAParams @Since("2.0.0") override def fit(dataset: Dataset[_]): PCAModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v} + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val pca = new feature.PCA(k = $(k)) val pcaModel = pca.fit(input) copyValues(new PCAModel(uid, pcaModel.pc, pcaModel.explainedVariance).setParent(this)) @@ -99,7 +107,7 @@ object PCA extends DefaultParamsReadable[PCA] { /** * :: Experimental :: - * Model fitted by [[PCA]]. + * Model fitted by [[PCA]]. Transforms vectors to a lower dimensional space. * * @param pc A principal components Matrix. Each column is one principal component. * @param explainedVariance A vector of proportions of variance explained by @@ -128,8 +136,14 @@ class PCAModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val pcaModel = new feature.PCAModel($(k), pc, explainedVariance) - val pcaOp = udf { pcaModel.transform _ } + val pcaModel = new feature.PCAModel($(k), + OldMatrices.fromML(pc).asInstanceOf[OldDenseMatrix], + OldVectors.fromML(explainedVariance).asInstanceOf[OldDenseVector]) + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer: Vector => Vector = v => pcaModel.transform(OldVectors.fromML(v)).asML + + val pcaOp = udf(transformer) dataset.withColumn($(outputCol), pcaOp(col($(inputCol)))) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index 0a9b9719c15d..a01867701bd8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -21,9 +21,9 @@ import scala.collection.mutable import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg._ import org.apache.spark.sql.types.DataType /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index 5219680be2dc..2916b6d9df3b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -25,10 +25,10 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model, Pipeline, PipelineModel, PipelineStage, Transformer} import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.ml.param.{Param, ParamMap} import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasLabelCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.VectorUDT import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types._ @@ -194,7 +194,9 @@ object RFormula extends DefaultParamsReadable[RFormula] { /** * :: Experimental :: - * A fitted RFormula. Fitting is required to determine the factor levels of formula terms. + * Model fitted by [[RFormula]]. Fitting is required to determine the factor levels of + * formula terms. + * * @param resolvedFormula the fitted R formula. * @param pipelineModel the fitted feature model, including factor to index mappings. */ @@ -256,8 +258,8 @@ class RFormulaModel private[feature]( val columnNames = schema.map(_.name) require(!columnNames.contains($(featuresCol)), "Features column already exists.") require( - !columnNames.contains($(labelCol)) || schema($(labelCol)).dataType == DoubleType, - "Label column already exists and is not of type DoubleType.") + !columnNames.contains($(labelCol)) || schema($(labelCol)).dataType.isInstanceOf[NumericType], + "Label column already exists and is not of type NumericType.") } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala index cf52710ab8cb..19aecff03885 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import scala.collection.mutable import scala.util.parsing.combinator.RegexParsers -import org.apache.spark.mllib.linalg.VectorUDT +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types._ /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala index 400435d7a937..2d4cac6dc42c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala @@ -48,6 +48,7 @@ class SQLTransformer @Since("1.6.0") (override val uid: String) extends Transfor /** * SQL statement parameter. The statement is provided in string form. + * * @group param */ @Since("1.6.0") @@ -66,7 +67,7 @@ class SQLTransformer @Since("1.6.0") (override val uid: String) extends Transfor @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { val tableName = Identifiable.randomUID(uid) - dataset.registerTempTable(tableName) + dataset.createOrReplaceTempView(tableName) val realStatement = $(statement).replace(tableIdentifier, tableName) dataset.sparkSession.sql(realStatement) } @@ -79,7 +80,7 @@ class SQLTransformer @Since("1.6.0") (override val uid: String) extends Transfor val dummyDF = sqlContext.createDataFrame(dummyRDD, schema) val tableName = Identifiable.randomUID(uid) val realStatement = $(statement).replace(tableIdentifier, tableName) - dummyDF.registerTempTable(tableName) + dummyDF.createOrReplaceTempView(tableName) val outputSchema = sqlContext.sql(realStatement).schema sqlContext.dropTempTable(tableName) outputSchema diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index 626e97efb47c..9d084b520c48 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -21,11 +21,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml._ +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -93,7 +96,9 @@ class StandardScaler(override val uid: String) extends Estimator[StandardScalerM @Since("2.0.0") override def fit(dataset: Dataset[_]): StandardScalerModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v } + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val scaler = new feature.StandardScaler(withMean = $(withMean), withStd = $(withStd)) val scalerModel = scaler.fit(input) copyValues(new StandardScalerModel(uid, scalerModel.std, scalerModel.mean).setParent(this)) @@ -145,7 +150,11 @@ class StandardScalerModel private[ml] ( override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) val scaler = new feature.StandardScalerModel(std, mean, $(withStd), $(withMean)) - val scale = udf { scaler.transform _ } + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer: Vector => Vector = v => scaler.transform(OldVectors.fromML(v)).asML + + val scale = udf(transformer) dataset.withColumn($(outputCol), scale(col($(inputCol)))) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala old mode 100644 new mode 100755 index b96bc48566fa..11864cb8f439 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala @@ -26,58 +26,6 @@ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{ArrayType, StringType, StructType} -/** - * stop words list - */ -private[spark] object StopWords { - - /** - * Use the same default stopwords list as scikit-learn. - * The original list can be found from "Glasgow Information Retrieval Group" - * [[http://ir.dcs.gla.ac.uk/resources/linguistic_utils/stop_words]] - */ - val English = Array( "a", "about", "above", "across", "after", "afterwards", "again", - "against", "all", "almost", "alone", "along", "already", "also", "although", "always", - "am", "among", "amongst", "amoungst", "amount", "an", "and", "another", - "any", "anyhow", "anyone", "anything", "anyway", "anywhere", "are", - "around", "as", "at", "back", "be", "became", "because", "become", - "becomes", "becoming", "been", "before", "beforehand", "behind", "being", - "below", "beside", "besides", "between", "beyond", "bill", "both", - "bottom", "but", "by", "call", "can", "cannot", "cant", "co", "con", - "could", "couldnt", "cry", "de", "describe", "detail", "do", "done", - "down", "due", "during", "each", "eg", "eight", "either", "eleven", "else", - "elsewhere", "empty", "enough", "etc", "even", "ever", "every", "everyone", - "everything", "everywhere", "except", "few", "fifteen", "fify", "fill", - "find", "fire", "first", "five", "for", "former", "formerly", "forty", - "found", "four", "from", "front", "full", "further", "get", "give", "go", - "had", "has", "hasnt", "have", "he", "hence", "her", "here", "hereafter", - "hereby", "herein", "hereupon", "hers", "herself", "him", "himself", "his", - "how", "however", "hundred", "i", "ie", "if", "in", "inc", "indeed", - "interest", "into", "is", "it", "its", "itself", "keep", "last", "latter", - "latterly", "least", "less", "ltd", "made", "many", "may", "me", - "meanwhile", "might", "mill", "mine", "more", "moreover", "most", "mostly", - "move", "much", "must", "my", "myself", "name", "namely", "neither", - "never", "nevertheless", "next", "nine", "no", "nobody", "none", "noone", - "nor", "not", "nothing", "now", "nowhere", "of", "off", "often", "on", - "once", "one", "only", "onto", "or", "other", "others", "otherwise", "our", - "ours", "ourselves", "out", "over", "own", "part", "per", "perhaps", - "please", "put", "rather", "re", "same", "see", "seem", "seemed", - "seeming", "seems", "serious", "several", "she", "should", "show", "side", - "since", "sincere", "six", "sixty", "so", "some", "somehow", "someone", - "something", "sometime", "sometimes", "somewhere", "still", "such", - "system", "take", "ten", "than", "that", "the", "their", "them", - "themselves", "then", "thence", "there", "thereafter", "thereby", - "therefore", "therein", "thereupon", "these", "they", "thick", "thin", - "third", "this", "those", "though", "three", "through", "throughout", - "thru", "thus", "to", "together", "too", "top", "toward", "towards", - "twelve", "twenty", "two", "un", "under", "until", "up", "upon", "us", - "very", "via", "was", "we", "well", "were", "what", "whatever", "when", - "whence", "whenever", "where", "whereafter", "whereas", "whereby", - "wherein", "whereupon", "wherever", "whether", "which", "while", "whither", - "who", "whoever", "whole", "whom", "whose", "why", "will", "with", - "within", "without", "would", "yet", "you", "your", "yours", "yourself", "yourselves") -} - /** * :: Experimental :: * A feature transformer that filters out stop words from input. @@ -97,11 +45,13 @@ class StopWordsRemover(override val uid: String) def setOutputCol(value: String): this.type = set(outputCol, value) /** - * the stop words set to be filtered out - * Default: [[StopWords.English]] + * The words to be filtered out. + * Default: English stop words + * @see [[StopWordsRemover.loadDefaultStopWords()]] * @group param */ - val stopWords: StringArrayParam = new StringArrayParam(this, "stopWords", "stop words") + val stopWords: StringArrayParam = + new StringArrayParam(this, "stopWords", "the words to be filtered out") /** @group setParam */ def setStopWords(value: Array[String]): this.type = set(stopWords, value) @@ -110,12 +60,12 @@ class StopWordsRemover(override val uid: String) def getStopWords: Array[String] = $(stopWords) /** - * whether to do a case sensitive comparison over the stop words + * Whether to do a case sensitive comparison over the stop words. * Default: false * @group param */ val caseSensitive: BooleanParam = new BooleanParam(this, "caseSensitive", - "whether to do case-sensitive comparison during filtering") + "whether to do a case-sensitive comparison over the stop words") /** @group setParam */ def setCaseSensitive(value: Boolean): this.type = set(caseSensitive, value) @@ -123,24 +73,24 @@ class StopWordsRemover(override val uid: String) /** @group getParam */ def getCaseSensitive: Boolean = $(caseSensitive) - setDefault(stopWords -> StopWords.English, caseSensitive -> false) + setDefault(stopWords -> StopWordsRemover.loadDefaultStopWords("english"), caseSensitive -> false) @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { val outputSchema = transformSchema(dataset.schema) val t = if ($(caseSensitive)) { - val stopWordsSet = $(stopWords).toSet - udf { terms: Seq[String] => - terms.filter(s => !stopWordsSet.contains(s)) - } - } else { - val toLower = (s: String) => if (s != null) s.toLowerCase else s - val lowerStopWords = $(stopWords).map(toLower(_)).toSet - udf { terms: Seq[String] => - terms.filter(s => !lowerStopWords.contains(toLower(s))) - } + val stopWordsSet = $(stopWords).toSet + udf { terms: Seq[String] => + terms.filter(s => !stopWordsSet.contains(s)) + } + } else { + // TODO: support user locale (SPARK-15064) + val toLower = (s: String) => if (s != null) s.toLowerCase else s + val lowerStopWords = $(stopWords).map(toLower(_)).toSet + udf { terms: Seq[String] => + terms.filter(s => !lowerStopWords.contains(toLower(s))) + } } - val metadata = outputSchema($(outputCol)).metadata dataset.select(col("*"), t(col($(inputCol))).as($(outputCol), metadata)) } @@ -158,6 +108,24 @@ class StopWordsRemover(override val uid: String) @Since("1.6.0") object StopWordsRemover extends DefaultParamsReadable[StopWordsRemover] { + private[feature] + val supportedLanguages = Set("danish", "dutch", "english", "finnish", "french", "german", + "hungarian", "italian", "norwegian", "portuguese", "russian", "spanish", "swedish", "turkish") + @Since("1.6.0") override def load(path: String): StopWordsRemover = super.load(path) + + /** + * Loads the default stop words for the given language. + * Supported languages: danish, dutch, english, finnish, french, german, hungarian, + * italian, norwegian, portuguese, russian, spanish, swedish, turkish + * @see [[http://anoncvs.postgresql.org/cvsweb.cgi/pgsql/src/backend/snowball/stopwords/]] + */ + @Since("2.0.0") + def loadDefaultStopWords(language: String): Array[String] = { + require(supportedLanguages.contains(language), + s"$language is not in the supported language list: ${supportedLanguages.mkString(", ")}.") + val is = getClass.getResourceAsStream(s"/org/apache/spark/ml/feature/stopwords/$language.txt") + scala.io.Source.fromInputStream(is)(scala.io.Codec.UTF8).getLines().toArray + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala index 4d3e46e488c6..1bc24202b761 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -23,10 +23,10 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute, UnresolvedAttribute} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala index 68b699d569c7..d814528ec48d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala @@ -27,10 +27,10 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, VectorUDT} import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.udf import org.apache.spark.sql.types.{StructField, StructType} @@ -240,7 +240,8 @@ object VectorIndexer extends DefaultParamsReadable[VectorIndexer] { /** * :: Experimental :: - * Transform categorical features to use 0-based indices instead of their original values. + * Model fitted by [[VectorIndexer]]. Transform categorical features to use 0-based indices + * instead of their original values. * - Categorical features are mapped to indices. * - Continuous features (columns) are left unchanged. * This also appends metadata to the output column, marking features as Numeric (continuous), diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala index 7a9468b87b73..103738cd91c0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala @@ -20,10 +20,10 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.{Attribute, AttributeGroup} +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{IntArrayParam, ParamMap, StringArrayParam} import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg._ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index c49e263df0a6..1469bfd5e857 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -22,11 +22,12 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkContext import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.sql.{DataFrame, Dataset, SQLContext} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala index 4571ab26800c..b94187ae787c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala @@ -44,7 +44,7 @@ import org.apache.spark.sql.DataFrame * import org.apache.spark.ml.Pipeline * * // a DataFrame with three columns: id (integer), text (string), and rating (double). - * val df = sqlContext.createDataFrame(Seq( + * val df = spark.createDataFrame(Seq( * (0, "Hi I heard about Spark", 3.0), * (1, "I wish Java could use case classes", 4.0), * (2, "Logistic regression models are neat", 4.0) diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala index c29f7f86e9f2..0b9b2ff5c5e2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.types._ * User-defined type for [[Vector]] in [[mllib-local]] which allows easy interaction with SQL * via [[org.apache.spark.sql.Dataset]]. */ -private[ml] class VectorUDT extends UserDefinedType[Vector] { +private[spark] class VectorUDT extends UserDefinedType[Vector] { override def sqlType: StructType = { // type: 0 = sparse, 1 = dense diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala index a2b52835e177..6ed193cf57d4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.optim import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance -import org.apache.spark.mllib.linalg._ +import org.apache.spark.ml.linalg._ import org.apache.spark.rdd.RDD /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala index 7d21302f962b..8f5f4427e1f4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala @@ -19,7 +19,8 @@ package org.apache.spark.ml.optim import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance -import org.apache.spark.mllib.linalg._ +import org.apache.spark.ml.linalg._ +import org.apache.spark.mllib.linalg.CholeskyDecomposition import org.apache.spark.rdd.RDD /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/package-info.java b/mllib/src/main/scala/org/apache/spark/ml/package-info.java index 87f4223964ad..9a40f5dd6913 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/package-info.java +++ b/mllib/src/main/scala/org/apache/spark/ml/package-info.java @@ -16,7 +16,7 @@ */ /** - * Spark ML is a BETA component that adds a new set of machine learning APIs to let users quickly + * Spark ML is a component that adds a new set of machine learning APIs to let users quickly * assemble and configure practical machine learning pipelines. */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/ml/package.scala b/mllib/src/main/scala/org/apache/spark/ml/package.scala index c589d06d9f7e..5cc328b4a9a1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/package.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/package.scala @@ -18,7 +18,7 @@ package org.apache.spark /** - * Spark ML is a BETA component that adds a new set of machine learning APIs to let users quickly + * Spark ML is a component that adds a new set of machine learning APIs to let users quickly * assemble and configure practical machine learning pipelines. * * @groupname param Parameters diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index c368aadd2366..82f2de7ccdfd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -29,8 +29,9 @@ import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.JsonVectorConverter import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * :: DeveloperApi :: @@ -92,7 +93,7 @@ class Param[T](val parent: String, val name: String, val doc: String, val isVali case x: String => compact(render(JString(x))) case v: Vector => - v.toJson + JsonVectorConverter.toJson(v) case _ => throw new NotImplementedError( "The default jsonEncode only supports string and vector. " + @@ -128,7 +129,7 @@ private[ml] object Param { val keys = v.map(_._1) assert(keys.contains("type") && keys.contains("values"), s"Expect a JSON serialized vector but cannot find fields 'type' and 'values' in $json.") - Vectors.fromJson(json).asInstanceOf[T] + JsonVectorConverter.fromJson(json).asInstanceOf[T] case _ => throw new NotImplementedError( "The default jsonDecode only supports string and vector. " + diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 541923048a3f..f257382d2205 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -30,7 +30,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.json4s.DefaultFormats import org.json4s.JsonDSL._ -import org.apache.spark.Partitioner +import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext} import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} @@ -42,7 +42,7 @@ import org.apache.spark.mllib.optimization.NNLS import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DoubleType, FloatType, IntegerType, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils import org.apache.spark.util.collection.{OpenHashMap, OpenHashSet, SortDataFormat, Sorter} @@ -53,24 +53,43 @@ import org.apache.spark.util.random.XORShiftRandom */ private[recommendation] trait ALSModelParams extends Params with HasPredictionCol { /** - * Param for the column name for user ids. + * Param for the column name for user ids. Ids must be integers. Other + * numeric types are supported for this column, but will be cast to integers as long as they + * fall within the integer value range. * Default: "user" * @group param */ - val userCol = new Param[String](this, "userCol", "column name for user ids") + val userCol = new Param[String](this, "userCol", "column name for user ids. Ids must be within " + + "the integer value range.") /** @group getParam */ def getUserCol: String = $(userCol) /** - * Param for the column name for item ids. + * Param for the column name for item ids. Ids must be integers. Other + * numeric types are supported for this column, but will be cast to integers as long as they + * fall within the integer value range. * Default: "item" * @group param */ - val itemCol = new Param[String](this, "itemCol", "column name for item ids") + val itemCol = new Param[String](this, "itemCol", "column name for item ids. Ids must be within " + + "the integer value range.") /** @group getParam */ def getItemCol: String = $(itemCol) + + /** + * Attempts to safely cast a user/item id to an Int. Throws an exception if the value is + * out of integer range. + */ + protected val checkedCast = udf { (n: Double) => + if (n > Int.MaxValue || n < Int.MinValue) { + throw new IllegalArgumentException(s"ALS only supports values in Integer range for columns " + + s"${$(userCol)} and ${$(itemCol)}. Value $n was out of Integer range.") + } else { + n.toInt + } + } } /** @@ -193,10 +212,11 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w * @return output schema */ protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(userCol), IntegerType) - SchemaUtils.checkColumnType(schema, $(itemCol), IntegerType) - val ratingType = schema($(ratingCol)).dataType - require(ratingType == FloatType || ratingType == DoubleType) + // user and item will be cast to Int + SchemaUtils.checkNumericType(schema, $(userCol)) + SchemaUtils.checkNumericType(schema, $(itemCol)) + // rating will be cast to Float + SchemaUtils.checkNumericType(schema, $(ratingCol)) SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) } } @@ -232,6 +252,7 @@ class ALSModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema) // Register a UDF for DataFrame, and then // create a new column named map(predictionCol) by running the predict UDF. val predict = udf { (userFeatures: Seq[Float], itemFeatures: Seq[Float]) => @@ -242,16 +263,19 @@ class ALSModel private[ml] ( } } dataset - .join(userFactors, dataset($(userCol)) === userFactors("id"), "left") - .join(itemFactors, dataset($(itemCol)) === itemFactors("id"), "left") + .join(userFactors, + checkedCast(dataset($(userCol)).cast(DoubleType)) === userFactors("id"), "left") + .join(itemFactors, + checkedCast(dataset($(itemCol)).cast(DoubleType)) === itemFactors("id"), "left") .select(dataset("*"), predict(userFactors("features"), itemFactors("features")).as($(predictionCol))) } @Since("1.3.0") override def transformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(userCol), IntegerType) - SchemaUtils.checkColumnType(schema, $(itemCol), IntegerType) + // user and item will be cast to Int + SchemaUtils.checkNumericType(schema, $(userCol)) + SchemaUtils.checkNumericType(schema, $(itemCol)) SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) } @@ -430,10 +454,13 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] @Since("2.0.0") override def fit(dataset: Dataset[_]): ALSModel = { + transformSchema(dataset.schema) import dataset.sparkSession.implicits._ + val r = if ($(ratingCol) != "") col($(ratingCol)).cast(FloatType) else lit(1.0f) val ratings = dataset - .select(col($(userCol)).cast(IntegerType), col($(itemCol)).cast(IntegerType), r) + .select(checkedCast(col($(userCol)).cast(DoubleType)), + checkedCast(col($(itemCol)).cast(DoubleType)), r) .rdd .map { row => Rating(row.getInt(0), row.getInt(1), row.getFloat(2)) @@ -706,13 +733,15 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { previousItemFactors.unpersist() itemFactors.setName(s"itemFactors-$iter").persist(intermediateRDDStorageLevel) // TODO: Generalize PeriodicGraphCheckpointer and use it here. + val deps = itemFactors.dependencies if (shouldCheckpoint(iter)) { - itemFactors.checkpoint() // itemFactors gets materialized in computeFactors. + itemFactors.checkpoint() // itemFactors gets materialized in computeFactors } val previousUserFactors = userFactors userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, itemLocalIndexEncoder, implicitPrefs, alpha, solver) if (shouldCheckpoint(iter)) { + ALS.cleanShuffleDependencies(sc, deps) deletePreviousCheckpointFile() previousCheckpointFile = itemFactors.getCheckpointFile } @@ -723,8 +752,10 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { itemFactors = computeFactors(userFactors, userOutBlocks, itemInBlocks, rank, regParam, userLocalIndexEncoder, solver = solver) if (shouldCheckpoint(iter)) { + val deps = itemFactors.dependencies itemFactors.checkpoint() itemFactors.count() // checkpoint item factors and cut lineage + ALS.cleanShuffleDependencies(sc, deps) deletePreviousCheckpointFile() previousCheckpointFile = itemFactors.getCheckpointFile } @@ -1355,4 +1386,31 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { * satisfies this requirement, we simply use a type alias here. */ private[recommendation] type ALSPartitioner = org.apache.spark.HashPartitioner + + /** + * Private function to clean up all of the shuffles files from the dependencies and their parents. + */ + private[spark] def cleanShuffleDependencies[T]( + sc: SparkContext, + deps: Seq[Dependency[_]], + blocking: Boolean = false): Unit = { + // If there is no reference tracking we skip clean up. + sc.cleaner.foreach { cleaner => + /** + * Clean the shuffles & all of its parents. + */ + def cleanEagerly(dep: Dependency[_]): Unit = { + if (dep.isInstanceOf[ShuffleDependency[_, _, _]]) { + val shuffleId = dep.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId + cleaner.doCleanupShuffle(shuffleId, blocking) + } + val rdd = dep.rdd + val rddDeps = rdd.dependencies + if (rdd.getStorageLevel == StorageLevel.NONE && rddDeps != null) { + rddDeps.foreach(cleanEagerly) + } + } + deps.foreach(cleanEagerly) + } + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 89ba6ab5d277..e63eb7108060 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -27,10 +27,11 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} @@ -88,8 +89,8 @@ private[regression] trait AFTSurvivalRegressionParams extends Params def getQuantilesCol: String = $(quantilesCol) /** Checks whether the input has quantiles column name. */ - protected[regression] def hasQuantilesCol: Boolean = { - isDefined(quantilesCol) && $(quantilesCol) != "" + private[regression] def hasQuantilesCol: Boolean = { + isDefined(quantilesCol) && $(quantilesCol).nonEmpty } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index 874d2a81db21..c4df9d11127f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -23,13 +23,13 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{PredictionModel, Predictor} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.DecisionTreeModelReadWrite._ import org.apache.spark.ml.tree.impl.RandomForest import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index c41fb4b0629b..81f2139f0b42 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -24,13 +24,13 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{PredictionModel, Predictor} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.GradientBoostedTrees import org.apache.spark.ml.util._ import org.apache.spark.ml.util.DefaultParamsReader.Metadata -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index c294ef31f90d..adbdd345e92e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -25,11 +25,11 @@ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{BLAS, Vector} import org.apache.spark.ml.optim._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{BLAS, Vector} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ @@ -43,6 +43,8 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam with HasFitIntercept with HasMaxIter with HasTol with HasRegParam with HasWeightCol with HasSolver with Logging { + import GeneralizedLinearRegression._ + /** * Param for the name of family which is a description of the error distribution * to be used in the model. @@ -54,8 +56,8 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam @Since("2.0.0") final val family: Param[String] = new Param(this, "family", "The name of family which is a description of the error distribution to be used in the " + - "model. Supported options: gaussian(default), binomial, poisson and gamma.", - ParamValidators.inArray[String](GeneralizedLinearRegression.supportedFamilyNames.toArray)) + s"model. Supported options: ${supportedFamilyNames.mkString(", ")}.", + ParamValidators.inArray[String](supportedFamilyNames.toArray)) /** @group getParam */ @Since("2.0.0") @@ -71,9 +73,8 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam @Since("2.0.0") final val link: Param[String] = new Param(this, "link", "The name of link function " + "which provides the relationship between the linear predictor and the mean of the " + - "distribution function. Supported options: identity, log, inverse, logit, probit, " + - "cloglog and sqrt.", - ParamValidators.inArray[String](GeneralizedLinearRegression.supportedLinkNames.toArray)) + s"distribution function. Supported options: ${supportedLinkNames.mkString(", ")}", + ParamValidators.inArray[String](supportedLinkNames.toArray)) /** @group getParam */ @Since("2.0.0") @@ -81,19 +82,23 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam /** * Param for link prediction (linear predictor) column name. - * Default is empty, which means we do not output link prediction. + * Default is not set, which means we do not output link prediction. * * @group param */ @Since("2.0.0") final val linkPredictionCol: Param[String] = new Param[String](this, "linkPredictionCol", "link prediction (linear predictor) column name") - setDefault(linkPredictionCol, "") /** @group getParam */ @Since("2.0.0") def getLinkPredictionCol: String = $(linkPredictionCol) + /** Checks whether we should output link prediction. */ + private[regression] def hasLinkPredictionCol: Boolean = { + isDefined(linkPredictionCol) && $(linkPredictionCol).nonEmpty + } + import GeneralizedLinearRegression._ @Since("2.0.0") @@ -107,7 +112,7 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam s"with ${$(family)} family does not support ${$(link)} link function.") } val newSchema = super.validateAndTransformSchema(schema, fitting, featuresDataType) - if ($(linkPredictionCol).nonEmpty) { + if (hasLinkPredictionCol) { SchemaUtils.appendColumn(newSchema, $(linkPredictionCol), DoubleType) } else { newSchema @@ -205,7 +210,7 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val /** * Sets the value of param [[weightCol]]. * If this is not set or empty, we treat all instance weights as 1.0. - * Default is empty, so all instances have weight one. + * Default is not set, so all instances have weight one. * * @group setParam */ @@ -214,7 +219,7 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val /** * Sets the solver algorithm used for optimization. - * Currently only support "irls" which is also the default solver. + * Currently only supports "irls" which is also the default solver. * * @group setParam */ @@ -239,10 +244,7 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val } val familyAndLink = new FamilyAndLink(familyObj, linkObj) - val numFeatures = dataset.select(col($(featuresCol))).limit(1).rdd - .map { case Row(features: Vector) => - features.size - }.first() + val numFeatures = dataset.select(col($(featuresCol))).first().getAs[Vector](0).size if (numFeatures > WeightedLeastSquares.MAX_NUM_FEATURES) { val msg = "Currently, GeneralizedLinearRegression only supports number of features" + s" <= ${WeightedLeastSquares.MAX_NUM_FEATURES}. Found $numFeatures in the input dataset." @@ -294,7 +296,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def load(path: String): GeneralizedLinearRegression = super.load(path) /** Set of family and link pairs that GeneralizedLinearRegression supports. */ - private[ml] lazy val supportedFamilyAndLinkPairs = Set( + private[regression] lazy val supportedFamilyAndLinkPairs = Set( Gaussian -> Identity, Gaussian -> Log, Gaussian -> Inverse, Binomial -> Logit, Binomial -> Probit, Binomial -> CLogLog, Poisson -> Log, Poisson -> Identity, Poisson -> Sqrt, @@ -302,17 +304,17 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine ) /** Set of family names that GeneralizedLinearRegression supports. */ - private[ml] lazy val supportedFamilyNames = supportedFamilyAndLinkPairs.map(_._1.name) + private[regression] lazy val supportedFamilyNames = supportedFamilyAndLinkPairs.map(_._1.name) /** Set of link names that GeneralizedLinearRegression supports. */ - private[ml] lazy val supportedLinkNames = supportedFamilyAndLinkPairs.map(_._2.name) + private[regression] lazy val supportedLinkNames = supportedFamilyAndLinkPairs.map(_._2.name) - private[ml] val epsilon: Double = 1E-16 + private[regression] val epsilon: Double = 1E-16 /** * Wrapper of family and link combination used in the model. */ - private[ml] class FamilyAndLink(val family: Family, val link: Link) extends Serializable { + private[regression] class FamilyAndLink(val family: Family, val link: Link) extends Serializable { /** Linear predictor based on given mu. */ def predict(mu: Double): Double = link.link(family.project(mu)) @@ -359,7 +361,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * * @param name the name of the family. */ - private[ml] abstract class Family(val name: String) extends Serializable { + private[regression] abstract class Family(val name: String) extends Serializable { /** The default link instance of this family. */ val defaultLink: Link @@ -391,7 +393,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine def project(mu: Double): Double = mu } - private[ml] object Family { + private[regression] object Family { /** * Gets the [[Family]] object from its name. @@ -412,7 +414,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * Gaussian exponential family distribution. * The default link for the Gaussian family is the identity link. */ - private[ml] object Gaussian extends Family("gaussian") { + private[regression] object Gaussian extends Family("gaussian") { val defaultLink: Link = Identity @@ -448,7 +450,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * Binomial exponential family distribution. * The default link for the Binomial family is the logit link. */ - private[ml] object Binomial extends Family("binomial") { + private[regression] object Binomial extends Family("binomial") { val defaultLink: Link = Logit @@ -492,7 +494,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * Poisson exponential family distribution. * The default link for the Poisson family is the log link. */ - private[ml] object Poisson extends Family("poisson") { + private[regression] object Poisson extends Family("poisson") { val defaultLink: Link = Log @@ -533,7 +535,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * Gamma exponential family distribution. * The default link for the Gamma family is the inverse link. */ - private[ml] object Gamma extends Family("gamma") { + private[regression] object Gamma extends Family("gamma") { val defaultLink: Link = Inverse @@ -578,7 +580,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * * @param name the name of link function. */ - private[ml] abstract class Link(val name: String) extends Serializable { + private[regression] abstract class Link(val name: String) extends Serializable { /** The link function. */ def link(mu: Double): Double @@ -590,7 +592,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine def unlink(eta: Double): Double } - private[ml] object Link { + private[regression] object Link { /** * Gets the [[Link]] object from its name. @@ -611,7 +613,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine } } - private[ml] object Identity extends Link("identity") { + private[regression] object Identity extends Link("identity") { override def link(mu: Double): Double = mu @@ -620,7 +622,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = eta } - private[ml] object Logit extends Link("logit") { + private[regression] object Logit extends Link("logit") { override def link(mu: Double): Double = math.log(mu / (1.0 - mu)) @@ -629,7 +631,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = 1.0 / (1.0 + math.exp(-1.0 * eta)) } - private[ml] object Log extends Link("log") { + private[regression] object Log extends Link("log") { override def link(mu: Double): Double = math.log(mu) @@ -638,7 +640,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = math.exp(eta) } - private[ml] object Inverse extends Link("inverse") { + private[regression] object Inverse extends Link("inverse") { override def link(mu: Double): Double = 1.0 / mu @@ -647,7 +649,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = 1.0 / eta } - private[ml] object Probit extends Link("probit") { + private[regression] object Probit extends Link("probit") { override def link(mu: Double): Double = dist.Gaussian(0.0, 1.0).icdf(mu) @@ -658,7 +660,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = dist.Gaussian(0.0, 1.0).cdf(eta) } - private[ml] object CLogLog extends Link("cloglog") { + private[regression] object CLogLog extends Link("cloglog") { override def link(mu: Double): Double = math.log(-1.0 * math.log(1 - mu)) @@ -667,7 +669,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = 1.0 - math.exp(-1.0 * math.exp(eta)) } - private[ml] object Sqrt extends Link("sqrt") { + private[regression] object Sqrt extends Link("sqrt") { override def link(mu: Double): Double = math.sqrt(mu) @@ -732,7 +734,7 @@ class GeneralizedLinearRegressionModel private[ml] ( if ($(predictionCol).nonEmpty) { output = output.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) } - if ($(linkPredictionCol).nonEmpty) { + if (hasLinkPredictionCol) { output = output.withColumn($(linkPredictionCol), predictLinkUDF(col($(featuresCol)))) } output.toDF() @@ -776,6 +778,13 @@ class GeneralizedLinearRegressionModel private[ml] ( .setParent(parent) } + /** + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. + * + * For [[GeneralizedLinearRegressionModel]], this does NOT currently save the + * training [[summary]]. An option to save [[summary]] may be added in the future. + * + */ @Since("2.0.0") override def write: MLWriter = new GeneralizedLinearRegressionModel.GeneralizedLinearRegressionModelWriter(this) @@ -848,12 +857,12 @@ class GeneralizedLinearRegressionSummary private[regression] ( import GeneralizedLinearRegression._ /** - * Field in "predictions" which gives the prediction value of each instance. + * Field in "predictions" which gives the predicted value of each instance. * This is set to a new column name if the original model's `predictionCol` is not set. */ @Since("2.0.0") val predictionCol: String = { - if (origModel.isDefined(origModel.predictionCol) && origModel.getPredictionCol != "") { + if (origModel.isDefined(origModel.predictionCol) && origModel.getPredictionCol.nonEmpty) { origModel.getPredictionCol } else { "prediction_" + java.util.UUID.randomUUID.toString @@ -870,7 +879,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( protected val model: GeneralizedLinearRegressionModel = origModel.copy(ParamMap.empty).setPredictionCol(predictionCol) - /** predictions output by the model's `transform` method */ + /** Predictions output by the model's `transform` method. */ @Since("2.0.0") @transient val predictions: DataFrame = model.transform(dataset) private[regression] lazy val family: Family = Family.fromName(model.getFamily) @@ -880,10 +889,10 @@ class GeneralizedLinearRegressionSummary private[regression] ( family.defaultLink } - /** Number of instances in DataFrame predictions */ + /** Number of instances in DataFrame predictions. */ private[regression] lazy val numInstances: Long = predictions.count() - /** The numeric rank of the fitted linear model */ + /** The numeric rank of the fitted linear model. */ @Since("2.0.0") lazy val rank: Long = if (model.getFitIntercept) { model.coefficients.size + 1 @@ -891,17 +900,17 @@ class GeneralizedLinearRegressionSummary private[regression] ( model.coefficients.size } - /** Degrees of freedom */ + /** Degrees of freedom. */ @Since("2.0.0") lazy val degreesOfFreedom: Long = { numInstances - rank } - /** The residual degrees of freedom */ + /** The residual degrees of freedom. */ @Since("2.0.0") lazy val residualDegreeOfFreedom: Long = degreesOfFreedom - /** The residual degrees of freedom for the null model */ + /** The residual degrees of freedom for the null model. */ @Since("2.0.0") lazy val residualDegreeOfFreedomNull: Long = if (model.getFitIntercept) { numInstances - 1 @@ -944,7 +953,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( } /** - * Get the default residuals(deviance residuals) of the fitted model. + * Get the default residuals (deviance residuals) of the fitted model. */ @Since("2.0.0") def residuals(): DataFrame = devianceResiduals @@ -1000,7 +1009,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( /** * The dispersion of the fitted model. * It is taken as 1.0 for the "binomial" and "poisson" families, and otherwise - * estimated by the residual Pearson's Chi-Squared statistic(which is defined as + * estimated by the residual Pearson's Chi-Squared statistic (which is defined as * sum of the squares of the Pearson residuals) divided by the residual degrees of freedom. */ @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala index 7a78ecbdf16d..d16e8e3f6b25 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala @@ -22,11 +22,11 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.regression.IsotonicRegressionModel.IsotonicRegressionModelWriter import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.mllib.regression.{IsotonicRegression => MLlibIsotonicRegression} import org.apache.spark.mllib.regression.{IsotonicRegressionModel => MLlibIsotonicRegressionModel} import org.apache.spark.rdd.RDD @@ -69,8 +69,8 @@ private[regression] trait IsotonicRegressionBase extends Params with HasFeatures setDefault(isotonic -> true, featureIndex -> 0) /** Checks whether the input has weight column. */ - protected[ml] def hasWeightCol: Boolean = { - isDefined(weightCol) && $(weightCol) != "" + private[regression] def hasWeightCol: Boolean = { + isDefined(weightCol) && $(weightCol).nonEmpty } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index d13b15fd82f0..ff1038cbf185 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -28,14 +28,16 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.BLAS._ import org.apache.spark.ml.optim.WeightedLeastSquares import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.evaluation.RegressionMetrics -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.linalg.BLAS._ +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} @@ -159,9 +161,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String override protected def train(dataset: Dataset[_]): LinearRegressionModel = { // Extract the number of features before deciding optimization solver. - val numFeatures = dataset.select(col($(featuresCol))).limit(1).rdd.map { - case Row(features: Vector) => features.size - }.first() + val numFeatures = dataset.select(col($(featuresCol))).first().getAs[Vector](0).size val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) if (($(solver) == "auto" && $(elasticNetParam) == 0.0 && @@ -240,7 +240,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String val coefficients = Vectors.sparse(numFeatures, Seq()) val intercept = yMean - val model = new LinearRegressionModel(uid, coefficients, intercept) + val model = copyValues(new LinearRegressionModel(uid, coefficients, intercept)) // Handle possible missing or invalid prediction columns val (summaryModel, predictionColName) = model.findSummaryModelAndPredictionCol() @@ -252,7 +252,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String model, Array(0D), Array(0D)) - return copyValues(model.setSummary(trainingSummary)) + return model.setSummary(trainingSummary) } else { require($(regParam) == 0.0, "The standard deviation of the label is zero. " + "Model cannot be regularized.") @@ -447,7 +447,7 @@ class LinearRegressionModel private[ml] ( } /** - * Returns a [[MLWriter]] instance for this ML instance. + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. * * For [[LinearRegressionModel]], this does NOT currently save the training [[summary]]. * An option to save [[summary]] may be added in the future. diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index 9605de72020f..a6dbf21d55e2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -22,13 +22,13 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{PredictionModel, Predictor} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.RandomForest import org.apache.spark.ml.util._ import org.apache.spark.ml.util.DefaultParamsReader.Metadata -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index ba2e1e2bc269..5ba768d55184 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -26,13 +26,13 @@ import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat import org.apache.spark.annotation.Since -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{DataFrame, DataFrameReader, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, JoinedRow} +import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ @@ -203,25 +203,18 @@ class DefaultSource extends FileFormat with DataSourceRegister { } val converter = RowEncoder(dataSchema) - - val unsafeRowIterator = points.map { pt => - val features = if (sparse) pt.features.toSparse else pt.features.toDense - converter.toRow(Row(pt.label, features)) - } - - def toAttribute(f: StructField): AttributeReference = + val fullOutput = dataSchema.map { f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)() - - // Appends partition values - val fullOutput = (dataSchema ++ partitionSchema).map(toAttribute) + } val requiredOutput = fullOutput.filter { a => - requiredSchema.fieldNames.contains(a.name) || partitionSchema.fieldNames.contains(a.name) + requiredSchema.fieldNames.contains(a.name) } - val joinedRow = new JoinedRow() - val appendPartitionColumns = GenerateUnsafeProjection.generate(requiredOutput, fullOutput) - unsafeRowIterator.map { dataRow => - appendPartitionColumns(joinedRow(dataRow, file.partitionValues)) + val requiredColumns = GenerateUnsafeProjection.generate(requiredOutput, fullOutput) + + points.map { pt => + val features = if (sparse) pt.features.toSparse else pt.features.toDense + requiredColumns(converter.toRow(Row(pt.label, features))) } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala index f71d28cf5953..d5e5c454605b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.tree import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector import org.apache.spark.mllib.tree.impurity.ImpurityCalculator import org.apache.spark.mllib.tree.model.{ImpurityStats, InformationGainStats => OldInformationGainStats, Node => OldNode, Predict => OldPredict} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala index a4287483d18e..9704e15cd838 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.tree import java.util.Objects import org.apache.spark.annotation.{DeveloperApi, Since} -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector import org.apache.spark.mllib.tree.configuration.{FeatureType => OldFeatureType} import org.apache.spark.mllib.tree.model.{Split => OldSplit} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala index 5f7c40f6071f..442f52bf0231 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala @@ -21,8 +21,8 @@ import scala.collection.mutable import scala.util.Try import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.tree.RandomForestParams -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ import org.apache.spark.mllib.tree.configuration.Strategy diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala index b6334762c7a7..a0faff236e9d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala @@ -18,10 +18,10 @@ package org.apache.spark.ml.tree.impl import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.regression.{DecisionTreeRegressionModel, DecisionTreeRegressor} import org.apache.spark.mllib.impl.PeriodicRDDCheckpointer -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.configuration.{BoostingStrategy => OldBoostingStrategy} import org.apache.spark.mllib.tree.impurity.{Variance => OldVariance} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index 2038a6873db7..be3792eb7732 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -24,10 +24,10 @@ import scala.util.Random import org.apache.spark.internal.Logging import org.apache.spark.ml.classification.DecisionTreeClassificationModel +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree._ import org.apache.spark.ml.util.Instrumentation -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.impurity.ImpurityCalculator import org.apache.spark.mllib.tree.model.ImpurityStats diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala index 3a2bf3c72573..a6ac64a0463c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala @@ -17,8 +17,8 @@ package org.apache.spark.ml.tree.impl +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.tree.{ContinuousSplit, Split} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala index f38e1ec7c09a..56c85c9b53e1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala @@ -23,11 +23,11 @@ import org.apache.hadoop.fs.Path import org.json4s._ import org.json4s.jackson.JsonMethods._ +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.{Param, Params} import org.apache.spark.ml.tree.DecisionTreeModelReadWrite.NodeData import org.apache.spark.ml.util.{DefaultParamsReader, DefaultParamsWriter} import org.apache.spark.ml.util.DefaultParamsReader.Metadata -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.tree.impurity.ImpurityCalculator import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index a41d02cde755..7d42da4a2ffa 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -30,7 +30,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.ml._ import org.apache.spark.ml.evaluation.Evaluator import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.HasSeed import org.apache.spark.ml.util._ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{DataFrame, Dataset} @@ -39,7 +38,7 @@ import org.apache.spark.sql.types.StructType /** * Params for [[CrossValidator]] and [[CrossValidatorModel]]. */ -private[ml] trait CrossValidatorParams extends ValidatorParams with HasSeed { +private[ml] trait CrossValidatorParams extends ValidatorParams { /** * Param for number of folds for cross validation. Must be >= 2. * Default: 3 @@ -179,11 +178,13 @@ object CrossValidator extends MLReadable[CrossValidator] { val (metadata, estimator, evaluator, estimatorParamMaps) = ValidatorParams.loadImpl(path, sc, className) val numFolds = (metadata.params \ "numFolds").extract[Int] + val seed = (metadata.params \ "seed").extract[Long] new CrossValidator(metadata.uid) .setEstimator(estimator) .setEvaluator(evaluator) .setEstimatorParamMaps(estimatorParamMaps) .setNumFolds(numFolds) + .setSeed(seed) } } } @@ -267,14 +268,16 @@ object CrossValidatorModel extends MLReadable[CrossValidatorModel] { val (metadata, estimator, evaluator, estimatorParamMaps) = ValidatorParams.loadImpl(path, sc, className) val numFolds = (metadata.params \ "numFolds").extract[Int] + val seed = (metadata.params \ "seed").extract[Long] val bestModelPath = new Path(path, "bestModel").toString val bestModel = DefaultParamsReader.loadParamsInstance[Model[_]](bestModelPath, sc) val avgMetrics = (metadata.metadata \ "avgMetrics").extract[Seq[Double]].toArray - val cv = new CrossValidatorModel(metadata.uid, bestModel, avgMetrics) - cv.set(cv.estimator, estimator) - .set(cv.evaluator, evaluator) - .set(cv.estimatorParamMaps, estimatorParamMaps) - .set(cv.numFolds, numFolds) + val model = new CrossValidatorModel(metadata.uid, bestModel, avgMetrics) + model.set(model.estimator, estimator) + .set(model.evaluator, evaluator) + .set(model.estimatorParamMaps, estimatorParamMaps) + .set(model.numFolds, numFolds) + .set(model.seed, seed) } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala index f2b7badbe513..f6f2bad401a1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala @@ -30,7 +30,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator import org.apache.spark.ml.param.{DoubleParam, ParamMap, ParamValidators} -import org.apache.spark.ml.param.shared.HasSeed import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types.StructType @@ -38,7 +37,7 @@ import org.apache.spark.sql.types.StructType /** * Params for [[TrainValidationSplit]] and [[TrainValidationSplitModel]]. */ -private[ml] trait TrainValidationSplitParams extends ValidatorParams with HasSeed { +private[ml] trait TrainValidationSplitParams extends ValidatorParams { /** * Param for ratio between train and validation data. Must be between 0 and 1. * Default: 0.75 @@ -177,11 +176,13 @@ object TrainValidationSplit extends MLReadable[TrainValidationSplit] { val (metadata, estimator, evaluator, estimatorParamMaps) = ValidatorParams.loadImpl(path, sc, className) val trainRatio = (metadata.params \ "trainRatio").extract[Double] + val seed = (metadata.params \ "seed").extract[Long] new TrainValidationSplit(metadata.uid) .setEstimator(estimator) .setEvaluator(evaluator) .setEstimatorParamMaps(estimatorParamMaps) .setTrainRatio(trainRatio) + .setSeed(seed) } } } @@ -265,14 +266,16 @@ object TrainValidationSplitModel extends MLReadable[TrainValidationSplitModel] { val (metadata, estimator, evaluator, estimatorParamMaps) = ValidatorParams.loadImpl(path, sc, className) val trainRatio = (metadata.params \ "trainRatio").extract[Double] + val seed = (metadata.params \ "seed").extract[Long] val bestModelPath = new Path(path, "bestModel").toString val bestModel = DefaultParamsReader.loadParamsInstance[Model[_]](bestModelPath, sc) val validationMetrics = (metadata.metadata \ "validationMetrics").extract[Seq[Double]].toArray - val tvs = new TrainValidationSplitModel(metadata.uid, bestModel, validationMetrics) - tvs.set(tvs.estimator, estimator) - .set(tvs.evaluator, evaluator) - .set(tvs.estimatorParamMaps, estimatorParamMaps) - .set(tvs.trainRatio, trainRatio) + val model = new TrainValidationSplitModel(metadata.uid, bestModel, validationMetrics) + model.set(model.estimator, estimator) + .set(model.evaluator, evaluator) + .set(model.estimatorParamMaps, estimatorParamMaps) + .set(model.trainRatio, trainRatio) + .set(model.seed, seed) } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala index 7a4e106aeb99..26fd73814d70 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala @@ -25,15 +25,15 @@ import org.apache.spark.SparkContext import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator import org.apache.spark.ml.param.{Param, ParamMap, ParamPair, Params} -import org.apache.spark.ml.util.{DefaultParamsReader, DefaultParamsWriter, MetaAlgorithmReadWrite, - MLWritable} +import org.apache.spark.ml.param.shared.HasSeed +import org.apache.spark.ml.util.{DefaultParamsReader, DefaultParamsWriter, MetaAlgorithmReadWrite, MLWritable} import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.sql.types.StructType /** * Common params for [[TrainValidationSplitParams]] and [[CrossValidatorParams]]. */ -private[ml] trait ValidatorParams extends Params { +private[ml] trait ValidatorParams extends HasSeed with Params { /** * param for the estimator to be validated @@ -137,7 +137,8 @@ private[ml] object ValidatorParams { } val jsonParams = validatorSpecificParams ++ List( - "estimatorParamMaps" -> parse(estimatorParamMapsJson)) + "estimatorParamMaps" -> parse(estimatorParamMapsJson), + "seed" -> parse(instance.seed.jsonEncode(instance.getSeed))) DefaultParamsWriter.saveMetadata(instance, path, sc, extraMetadata, Some(jsonParams)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala index 96a38a3bde96..f34a8310ddf1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.util import scala.collection.immutable.HashMap import org.apache.spark.ml.attribute._ -import org.apache.spark.mllib.linalg.VectorUDT +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types.StructField diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala b/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala index 8d4174124b5c..e79b1f31643d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala @@ -19,7 +19,8 @@ package org.apache.spark.ml.util import scala.collection.mutable -import org.apache.spark.{Accumulator, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.util.LongAccumulator; /** * Abstract class for stopwatches. @@ -102,12 +103,12 @@ private[spark] class DistributedStopwatch( sc: SparkContext, override val name: String) extends Stopwatch { - private val elapsedTime: Accumulator[Long] = sc.accumulator(0L, s"DistributedStopwatch($name)") + private val elapsedTime: LongAccumulator = sc.longAccumulator(s"DistributedStopwatch($name)") override def elapsed(): Long = elapsedTime.value override protected def add(duration: Long): Unit = { - elapsedTime += duration + elapsedTime.add(duration) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 8daee7b3aa1e..90d382753131 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -30,6 +30,8 @@ import net.razorvine.pickle._ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.python.SerDeUtil +import org.apache.spark.ml.feature.{LabeledPoint => MLLabeledPoint} +import org.apache.spark.ml.linalg.{DenseMatrix => NewDenseMatrix, DenseVector => NewDenseVector, SparseMatrix => NewSparseMatrix, SparseVector => NewSparseVector, Vector => NewVector, Vectors => NewVectors} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.evaluation.RankingMetrics @@ -1207,6 +1209,7 @@ private[python] class PythonMLLibAPI extends Serializable { private[spark] object SerDe extends Serializable { val PYSPARK_PACKAGE = "pyspark.mllib" + val PYSPARK_ML_PACKAGE = "pyspark.ml" /** * Base class used for pickle @@ -1214,8 +1217,10 @@ private[spark] object SerDe extends Serializable { private[python] abstract class BasePickler[T: ClassTag] extends IObjectPickler with IObjectConstructor { + protected def packageName: String = PYSPARK_PACKAGE + private val cls = implicitly[ClassTag[T]].runtimeClass - private val module = PYSPARK_PACKAGE + "." + cls.getName.split('.')(4) + private val module = packageName + "." + cls.getName.split('.')(4) private val name = cls.getSimpleName // register this to Pickler and Unpickler @@ -1262,7 +1267,7 @@ private[spark] object SerDe extends Serializable { private[python] def saveState(obj: Object, out: OutputStream, pickler: Pickler) } - // Pickler for DenseVector + // Pickler for (mllib) DenseVector private[python] class DenseVectorPickler extends BasePickler[DenseVector] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1294,7 +1299,41 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for DenseMatrix + // Pickler for (new) DenseVector + private[python] class NewDenseVectorPickler extends BasePickler[NewDenseVector] { + + override protected def packageName = PYSPARK_ML_PACKAGE + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val vector: NewDenseVector = obj.asInstanceOf[NewDenseVector] + val bytes = new Array[Byte](8 * vector.size) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + val db = bb.asDoubleBuffer() + db.put(vector.values) + + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(bytes.length)) + out.write(bytes) + out.write(Opcodes.TUPLE1) + } + + def construct(args: Array[Object]): Object = { + require(args.length == 1) + if (args.length != 1) { + throw new PickleException("should be 1") + } + val bytes = getBytes(args(0)) + val bb = ByteBuffer.wrap(bytes, 0, bytes.length) + bb.order(ByteOrder.nativeOrder()) + val db = bb.asDoubleBuffer() + val ans = new Array[Double](bytes.length / 8) + db.get(ans) + NewVectors.dense(ans) + } + } + + // Pickler for (mllib) DenseMatrix private[python] class DenseMatrixPickler extends BasePickler[DenseMatrix] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1331,7 +1370,46 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for SparseMatrix + // Pickler for (new) DenseMatrix + private[python] class NewDenseMatrixPickler extends BasePickler[NewDenseMatrix] { + + override protected def packageName = PYSPARK_ML_PACKAGE + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val m: NewDenseMatrix = obj.asInstanceOf[NewDenseMatrix] + val bytes = new Array[Byte](8 * m.values.length) + val order = ByteOrder.nativeOrder() + val isTransposed = if (m.isTransposed) 1 else 0 + ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().put(m.values) + + out.write(Opcodes.MARK) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(m.numRows)) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(m.numCols)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(bytes.length)) + out.write(bytes) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(isTransposed)) + out.write(Opcodes.TUPLE) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 4) { + throw new PickleException("should be 4") + } + val bytes = getBytes(args(2)) + val n = bytes.length / 8 + val values = new Array[Double](n) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().get(values) + val isTransposed = args(3).asInstanceOf[Int] == 1 + new NewDenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], values, isTransposed) + } + } + + // Pickler for (mllib) SparseMatrix private[python] class SparseMatrixPickler extends BasePickler[SparseMatrix] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1386,7 +1464,64 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for SparseVector + // Pickler for (new) SparseMatrix + private[python] class NewSparseMatrixPickler extends BasePickler[NewSparseMatrix] { + + override protected def packageName = PYSPARK_ML_PACKAGE + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val s = obj.asInstanceOf[NewSparseMatrix] + val order = ByteOrder.nativeOrder() + + val colPtrsBytes = new Array[Byte](4 * s.colPtrs.length) + val indicesBytes = new Array[Byte](4 * s.rowIndices.length) + val valuesBytes = new Array[Byte](8 * s.values.length) + val isTransposed = if (s.isTransposed) 1 else 0 + ByteBuffer.wrap(colPtrsBytes).order(order).asIntBuffer().put(s.colPtrs) + ByteBuffer.wrap(indicesBytes).order(order).asIntBuffer().put(s.rowIndices) + ByteBuffer.wrap(valuesBytes).order(order).asDoubleBuffer().put(s.values) + + out.write(Opcodes.MARK) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(s.numRows)) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(s.numCols)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(colPtrsBytes.length)) + out.write(colPtrsBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(indicesBytes.length)) + out.write(indicesBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(valuesBytes.length)) + out.write(valuesBytes) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(isTransposed)) + out.write(Opcodes.TUPLE) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 6) { + throw new PickleException("should be 6") + } + val order = ByteOrder.nativeOrder() + val colPtrsBytes = getBytes(args(2)) + val indicesBytes = getBytes(args(3)) + val valuesBytes = getBytes(args(4)) + val colPtrs = new Array[Int](colPtrsBytes.length / 4) + val rowIndices = new Array[Int](indicesBytes.length / 4) + val values = new Array[Double](valuesBytes.length / 8) + ByteBuffer.wrap(colPtrsBytes).order(order).asIntBuffer().get(colPtrs) + ByteBuffer.wrap(indicesBytes).order(order).asIntBuffer().get(rowIndices) + ByteBuffer.wrap(valuesBytes).order(order).asDoubleBuffer().get(values) + val isTransposed = args(5).asInstanceOf[Int] == 1 + new NewSparseMatrix( + args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], colPtrs, rowIndices, values, + isTransposed) + } + } + + // Pickler for (mllib) SparseVector private[python] class SparseVectorPickler extends BasePickler[SparseVector] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1428,7 +1563,51 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for LabeledPoint + // Pickler for (new) SparseVector + private[python] class NewSparseVectorPickler extends BasePickler[NewSparseVector] { + + override protected def packageName = PYSPARK_ML_PACKAGE + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val v: NewSparseVector = obj.asInstanceOf[NewSparseVector] + val n = v.indices.length + val indiceBytes = new Array[Byte](4 * n) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().put(v.indices) + val valueBytes = new Array[Byte](8 * n) + ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().put(v.values) + + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(v.size)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(indiceBytes.length)) + out.write(indiceBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(valueBytes.length)) + out.write(valueBytes) + out.write(Opcodes.TUPLE3) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 3) { + throw new PickleException("should be 3") + } + val size = args(0).asInstanceOf[Int] + val indiceBytes = getBytes(args(1)) + val valueBytes = getBytes(args(2)) + val n = indiceBytes.length / 4 + val indices = new Array[Int](n) + val values = new Array[Double](n) + if (n > 0) { + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().get(indices) + ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().get(values) + } + new NewSparseVector(size, indices, values) + } + } + + // Pickler for MLlib LabeledPoint private[python] class LabeledPointPickler extends BasePickler[LabeledPoint] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1482,6 +1661,10 @@ private[spark] object SerDe extends Serializable { new DenseMatrixPickler().register() new SparseMatrixPickler().register() new SparseVectorPickler().register() + new NewDenseVectorPickler().register() + new NewDenseMatrixPickler().register() + new NewSparseMatrixPickler().register() + new NewSparseVectorPickler().register() new LabeledPointPickler().register() new RatingPickler().register() initialized = true diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 1d25a58e0f2e..f3c52f61a3bb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -431,7 +431,7 @@ class LogisticRegressionWithLBFGS if (userSuppliedWeights) { val uid = Identifiable.randomUID("logreg-static") lr.setInitialModel(new org.apache.spark.ml.classification.LogisticRegressionModel( - uid, initialWeights, 1.0)) + uid, initialWeights.asML, 1.0)) } lr.setFitIntercept(addIntercept) lr.setMaxIter(optimizer.getNumIterations()) @@ -439,7 +439,7 @@ class LogisticRegressionWithLBFGS // Convert our input into a DataFrame val sqlContext = new SQLContext(input.context) import sqlContext.implicits._ - val df = input.toDF() + val df = input.map(_.asML).toDF() // Determine if we should cache the DF val handlePersistence = input.getStorageLevel == StorageLevel.NONE // Train our model diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 60f13d27d0a6..38728f2693c6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -279,7 +279,7 @@ class KMeans private ( } val activeCenters = activeRuns.map(r => centers(r)).toArray - val costAccums = activeRuns.map(_ => sc.accumulator(0.0)) + val costAccums = activeRuns.map(_ => sc.doubleAccumulator) val bcActiveCenters = sc.broadcast(activeCenters) @@ -296,7 +296,7 @@ class KMeans private ( points.foreach { point => (0 until runs).foreach { i => val (bestCenter, cost) = KMeans.findClosest(thisActiveCenters(i), point) - costAccums(i) += cost + costAccums(i).add(cost) val sum = sums(i)(bestCenter) axpy(1.0, point.vector, sum) counts(i)(bestCenter) += 1 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala index adf20dc4b8b1..53587670a5db 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala @@ -46,17 +46,15 @@ private[mllib] object LocalKMeans extends Logging { // Initialize centers by sampling using the k-means++ procedure. centers(0) = pickWeighted(rand, points, weights).toDense + val costArray = points.map(KMeans.fastSquaredDistance(_, centers(0))) + for (i <- 1 until k) { - // Pick the next center with a probability proportional to cost under current centers - val curCenters = centers.view.take(i) - val sum = points.view.zip(weights).map { case (p, w) => - w * KMeans.pointCost(curCenters, p) - }.sum + val sum = costArray.zip(weights).map(p => p._1 * p._2).sum val r = rand.nextDouble() * sum var cumulativeScore = 0.0 var j = 0 while (j < points.length && cumulativeScore < r) { - cumulativeScore += weights(j) * KMeans.pointCost(curCenters, points(j)) + cumulativeScore += weights(j) * costArray(j) j += 1 } if (j == 0) { @@ -66,6 +64,12 @@ private[mllib] object LocalKMeans extends Logging { } else { centers(i) = points(j - 1).toDense } + + // update costArray + for (p <- points.indices) { + costArray(p) = math.min(KMeans.fastSquaredDistance(points(p), centers(i)), costArray(p)) + } + } // Run up to maxIterations iterations of Lloyd's algorithm diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala index 5dde2bdb17f3..719695a3380d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala @@ -139,7 +139,8 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl * Returns precision */ @Since("1.1.0") - lazy val precision: Double = tpByClass.values.sum.toDouble / labelCount + @deprecated("Use accuracy.", "2.0.0") + lazy val precision: Double = accuracy /** * Returns recall @@ -148,14 +149,24 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl * of all false negatives) */ @Since("1.1.0") - lazy val recall: Double = precision + @deprecated("Use accuracy.", "2.0.0") + lazy val recall: Double = accuracy /** * Returns f-measure * (equals to precision and recall because precision equals recall) */ @Since("1.1.0") - lazy val fMeasure: Double = precision + @deprecated("Use accuracy.", "2.0.0") + lazy val fMeasure: Double = accuracy + + /** + * Returns accuracy + * (equals to the total number of correctly classified instances + * out of the total number of instances.) + */ + @Since("2.0.0") + lazy val accuracy: Double = tpByClass.values.sum.toDouble / labelCount /** * Returns weighted true positive rate diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index 9166faa54de5..28e4966f918a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -116,7 +116,7 @@ object FPGrowthModel extends Loader[FPGrowthModel[_]] { StructField("freq", LongType)) val schema = StructType(fields) val rowDataRDD = model.freqItemsets.map { x => - Row(x.items, x.freq) + Row(x.items.toSeq, x.freq) } sqlContext.createDataFrame(rowDataRDD, schema).write.parquet(Loader.dataPath(path)) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 076cca6016ec..ee1956c2d413 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.linalg import java.util.{Arrays, Random} import scala.collection.mutable.{ArrayBuffer, ArrayBuilder => MArrayBuilder, HashSet => MHashSet} +import scala.language.implicitConversions import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM} import com.github.fommil.netlib.BLAS.{getInstance => blas} @@ -163,7 +164,8 @@ sealed trait Matrix extends Serializable { * Convert this matrix to the new mllib-local representation. * This does NOT copy the data; it copies references. */ - private[spark] def asML: newlinalg.Matrix + @Since("2.0.0") + def asML: newlinalg.Matrix } private[spark] class MatrixUDT extends UserDefinedType[Matrix] { @@ -426,7 +428,8 @@ class DenseMatrix @Since("1.3.0") ( } } - private[spark] override def asML: newlinalg.DenseMatrix = { + @Since("2.0.0") + override def asML: newlinalg.DenseMatrix = { new newlinalg.DenseMatrix(numRows, numCols, values, isTransposed) } } @@ -526,8 +529,11 @@ object DenseMatrix { matrix } - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(m: newlinalg.DenseMatrix): DenseMatrix = { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(m: newlinalg.DenseMatrix): DenseMatrix = { new DenseMatrix(m.numRows, m.numCols, m.values, m.isTransposed) } } @@ -739,7 +745,8 @@ class SparseMatrix @Since("1.3.0") ( } } - private[spark] override def asML: newlinalg.SparseMatrix = { + @Since("2.0.0") + override def asML: newlinalg.SparseMatrix = { new newlinalg.SparseMatrix(numRows, numCols, colPtrs, rowIndices, values, isTransposed) } } @@ -917,8 +924,11 @@ object SparseMatrix { } } - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(m: newlinalg.SparseMatrix): SparseMatrix = { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(m: newlinalg.SparseMatrix): SparseMatrix = { new SparseMatrix(m.numRows, m.numCols, m.colPtrs, m.rowIndices, m.values, m.isTransposed) } } @@ -1204,11 +1214,35 @@ object Matrices { } } - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(m: newlinalg.Matrix): Matrix = m match { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(m: newlinalg.Matrix): Matrix = m match { case dm: newlinalg.DenseMatrix => DenseMatrix.fromML(dm) case sm: newlinalg.SparseMatrix => SparseMatrix.fromML(sm) } } + +/** + * Implicit methods available in Scala for converting [[org.apache.spark.mllib.linalg.Matrix]] to + * [[org.apache.spark.ml.linalg.Matrix]] and vice versa. + */ +private[spark] object MatrixImplicits { + + implicit def mllibMatrixToMLMatrix(m: Matrix): newlinalg.Matrix = m.asML + + implicit def mllibDenseMatrixToMLDenseMatrix(m: DenseMatrix): newlinalg.DenseMatrix = m.asML + + implicit def mllibSparseMatrixToMLSparseMatrix(m: SparseMatrix): newlinalg.SparseMatrix = m.asML + + implicit def mlMatrixToMLlibMatrix(m: newlinalg.Matrix): Matrix = Matrices.fromML(m) + + implicit def mlDenseMatrixToMLlibDenseMatrix(m: newlinalg.DenseMatrix): DenseMatrix = + Matrices.fromML(m).asInstanceOf[DenseMatrix] + + implicit def mlSparseMatrixToMLlibSparseMatrix(m: newlinalg.SparseMatrix): SparseMatrix = + Matrices.fromML(m).asInstanceOf[SparseMatrix] +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 132e54a8c3de..7ebcd297bd54 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -22,6 +22,7 @@ import java.util import scala.annotation.varargs import scala.collection.JavaConverters._ +import scala.language.implicitConversions import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} import org.json4s.DefaultFormats @@ -185,7 +186,8 @@ sealed trait Vector extends Serializable { * Convert this vector to the new mllib-local representation. * This does NOT copy the data; it copies references. */ - private[spark] def asML: newlinalg.Vector + @Since("2.0.0") + def asML: newlinalg.Vector } /** @@ -580,8 +582,11 @@ object Vectors { /** Max number of nonzero entries used in computing hash code. */ private[linalg] val MAX_HASH_NNZ = 128 - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(v: newlinalg.Vector): Vector = v match { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(v: newlinalg.Vector): Vector = v match { case dv: newlinalg.DenseVector => DenseVector.fromML(dv) case sv: newlinalg.SparseVector => @@ -703,7 +708,8 @@ class DenseVector @Since("1.0.0") ( compact(render(jValue)) } - private[spark] override def asML: newlinalg.DenseVector = { + @Since("2.0.0") + override def asML: newlinalg.DenseVector = { new newlinalg.DenseVector(values) } } @@ -715,8 +721,11 @@ object DenseVector { @Since("1.3.0") def unapply(dv: DenseVector): Option[Array[Double]] = Some(dv.values) - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(v: newlinalg.DenseVector): DenseVector = { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(v: newlinalg.DenseVector): DenseVector = { new DenseVector(v.values) } } @@ -910,7 +919,8 @@ class SparseVector @Since("1.0.0") ( compact(render(jValue)) } - private[spark] override def asML: newlinalg.SparseVector = { + @Since("2.0.0") + override def asML: newlinalg.SparseVector = { new newlinalg.SparseVector(size, indices, values) } } @@ -921,8 +931,32 @@ object SparseVector { def unapply(sv: SparseVector): Option[(Int, Array[Int], Array[Double])] = Some((sv.size, sv.indices, sv.values)) - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(v: newlinalg.SparseVector): SparseVector = { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(v: newlinalg.SparseVector): SparseVector = { new SparseVector(v.size, v.indices, v.values) } } + +/** + * Implicit methods available in Scala for converting [[org.apache.spark.mllib.linalg.Vector]] to + * [[org.apache.spark.ml.linalg.Vector]] and vice versa. + */ +private[spark] object VectorImplicits { + + implicit def mllibVectorToMLVector(v: Vector): newlinalg.Vector = v.asML + + implicit def mllibDenseVectorToMLDenseVector(v: DenseVector): newlinalg.DenseVector = v.asML + + implicit def mllibSparseVectorToMLSparseVector(v: SparseVector): newlinalg.SparseVector = v.asML + + implicit def mlVectorToMLlibVector(v: newlinalg.Vector): Vector = Vectors.fromML(v) + + implicit def mlDenseVectorToMLlibDenseVector(v: newlinalg.DenseVector): DenseVector = + Vectors.fromML(v).asInstanceOf[DenseVector] + + implicit def mlSparseVectorToMLlibSparseVector(v: newlinalg.SparseVector): SparseVector = + Vectors.fromML(v).asInstanceOf[SparseVector] +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala index 45540f0c5c4c..f082b16b95e8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.regression import scala.beans.BeanInfo import org.apache.spark.annotation.Since +import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.NumericParser import org.apache.spark.SparkException @@ -38,6 +39,10 @@ case class LabeledPoint @Since("1.0.0") ( override def toString: String = { s"($label,$features)" } + + private[spark] def asML: NewLabeledPoint = { + NewLabeledPoint(label, features.asML) + } } /** @@ -67,4 +72,8 @@ object LabeledPoint { LabeledPoint(label, features) } } + + private[spark] def fromML(point: NewLabeledPoint): LabeledPoint = { + LabeledPoint(point.label, Vectors.fromML(point.features)) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala index 6c6e9fb7c6b3..c806d6bdf6b6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala @@ -64,7 +64,7 @@ class MultivariateGaussian @Since("1.3.0") ( /** * Returns density of this multivariate Gaussian at given point, x */ - @Since("1.3.0") + @Since("1.3.0") def pdf(x: Vector): Double = { pdf(x.toBreeze) } @@ -72,7 +72,7 @@ class MultivariateGaussian @Since("1.3.0") ( /** * Returns the log-density of this multivariate Gaussian at given point, x */ - @Since("1.3.0") + @Since("1.3.0") def logpdf(x: Vector): Double = { logpdf(x.toBreeze) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index 7fe60e2d99e4..ece1e41d986d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.tree import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} import org.apache.spark.ml.tree.impl.{GradientBoostedTrees => NewGBT} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.BoostingStrategy @@ -66,7 +67,9 @@ class GradientBoostedTrees private[spark] ( @Since("1.2.0") def run(input: RDD[LabeledPoint]): GradientBoostedTreesModel = { val algo = boostingStrategy.treeStrategy.algo - val (trees, treeWeights) = NewGBT.run(input, boostingStrategy, seed.toLong) + val (trees, treeWeights) = NewGBT.run(input.map { point => + NewLabeledPoint(point.label, point.features.asML) + }, boostingStrategy, seed.toLong) new GradientBoostedTreesModel(algo, trees.map(_.toOld), treeWeights) } @@ -94,8 +97,11 @@ class GradientBoostedTrees private[spark] ( input: RDD[LabeledPoint], validationInput: RDD[LabeledPoint]): GradientBoostedTreesModel = { val algo = boostingStrategy.treeStrategy.algo - val (trees, treeWeights) = NewGBT.runWithValidation(input, validationInput, boostingStrategy, - seed.toLong) + val (trees, treeWeights) = NewGBT.runWithValidation(input.map { point => + NewLabeledPoint(point.label, point.features.asML) + }, validationInput.map { point => + NewLabeledPoint(point.label, point.features.asML) + }, boostingStrategy, seed.toLong) new GradientBoostedTreesModel(algo, trees.map(_.toOld), treeWeights) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index c31ed9c1ce0b..14f11ce51b87 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -90,8 +90,8 @@ private class RandomForest ( * @return RandomForestModel that can be used for prediction. */ def run(input: RDD[LabeledPoint]): RandomForestModel = { - val trees: Array[NewDTModel] = - NewRandomForest.run(input, strategy, numTrees, featureSubsetStrategy, seed.toLong, None) + val trees: Array[NewDTModel] = NewRandomForest.run(input.map(_.asML), strategy, numTrees, + featureSubsetStrategy, seed.toLong, None) new RandomForestModel(strategy.algo, trees.map(_.toOld)) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index a87f8a6cde31..c13b9a66c4e2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -75,8 +75,8 @@ class DecisionTreeModel @Since("1.0.0") ( * @return JavaRDD of predictions for each of the given data points */ @Since("1.2.0") - def predict(features: JavaRDD[Vector]): JavaRDD[Double] = { - predict(features.rdd) + def predict(features: JavaRDD[Vector]): JavaRDD[java.lang.Double] = { + predict(features.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Double]] } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 774170ff401e..f0346e600d23 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -104,7 +104,7 @@ object MLUtils { val (indices, values) = items.tail.filter(_.nonEmpty).map { item => val indexAndValue = item.split(':') val index = indexAndValue(0).toInt - 1 // Convert 1-based indices to 0-based. - val value = indexAndValue(1).toDouble + val value = indexAndValue(1).toDouble (index, value) }.unzip @@ -119,7 +119,6 @@ object MLUtils { previous = current i += 1 } - (label, indices.toArray, values.toArray) } diff --git a/mllib/src/test/java/org/apache/spark/SharedSparkSession.java b/mllib/src/test/java/org/apache/spark/SharedSparkSession.java new file mode 100644 index 000000000000..43779878890d --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/SharedSparkSession.java @@ -0,0 +1,48 @@ +/* + * 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; + +import java.io.IOException; +import java.io.Serializable; + +import org.junit.After; +import org.junit.Before; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; + +public abstract class SharedSparkSession implements Serializable { + + protected transient SparkSession spark; + protected transient JavaSparkContext jsc; + + @Before + public void setUp() throws IOException { + spark = SparkSession.builder() + .master("local[2]") + .appName(getClass().getSimpleName()) + .getOrCreate(); + jsc = new JavaSparkContext(spark.sparkContext()); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } +} diff --git a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java index 60a4a1d2ea2a..9b209006bc36 100644 --- a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java @@ -17,42 +17,32 @@ package org.apache.spark.ml; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.junit.After; -import org.junit.Before; +import java.io.IOException; + import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.ml.classification.LogisticRegression; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.feature.StandardScaler; -import org.apache.spark.sql.SQLContext; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; /** * Test Pipeline construction and fitting in Java. */ -public class JavaPipelineSuite { +public class JavaPipelineSuite extends SharedSparkSession { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; private transient Dataset dataset; - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaPipelineSuite"); - jsql = new SQLContext(jsc); + @Override + public void setUp() throws IOException { + super.setUp(); JavaRDD points = jsc.parallelize(generateLogisticInputAsList(1.0, 1.0, 100, 42), 2); - dataset = jsql.createDataFrame(points, LabeledPoint.class); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; + dataset = spark.createDataFrame(points, LabeledPoint.class); } @Test @@ -63,10 +53,10 @@ public void pipeline() { LogisticRegression lr = new LogisticRegression() .setFeaturesCol("scaledFeatures"); Pipeline pipeline = new Pipeline() - .setStages(new PipelineStage[] {scaler, lr}); + .setStages(new PipelineStage[]{scaler, lr}); PipelineModel model = pipeline.fit(dataset); - model.transform(dataset).registerTempTable("prediction"); - Dataset predictions = jsql.sql("SELECT label, probability, prediction FROM prediction"); + model.transform(dataset).createOrReplaceTempView("prediction"); + Dataset predictions = spark.sql("SELECT label, probability, prediction FROM prediction"); predictions.collectAsList(); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/attribute/JavaAttributeSuite.java b/mllib/src/test/java/org/apache/spark/ml/attribute/JavaAttributeSuite.java index b74bbed23143..15cde0d3c045 100644 --- a/mllib/src/test/java/org/apache/spark/ml/attribute/JavaAttributeSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/attribute/JavaAttributeSuite.java @@ -17,8 +17,8 @@ package org.apache.spark.ml.attribute; -import org.junit.Test; import org.junit.Assert; +import org.junit.Test; public class JavaAttributeSuite { diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java index 1f2368262159..5aba4e8f7de0 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java @@ -17,37 +17,19 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; - - -public class JavaDecisionTreeClassifierSuite implements Serializable { - - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaDecisionTreeClassifierSuite"); - } +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaDecisionTreeClassifierSuite extends SharedSparkSession { @Test public void runDT() { @@ -55,7 +37,7 @@ public void runDT() { double A = 2.0; double B = -1.5; - JavaRDD data = sc.parallelize( + JavaRDD data = jsc.parallelize( LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); Map categoricalFeatures = new HashMap<>(); Dataset dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); @@ -70,7 +52,7 @@ public void runDT() { .setCacheNodeIds(false) .setCheckpointInterval(10) .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern - for (String impurity: DecisionTreeClassifier.supportedImpurities()) { + for (String impurity : DecisionTreeClassifier.supportedImpurities()) { dt.setImpurity(impurity); } DecisionTreeClassificationModel model = dt.fit(dataFrame); diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java index 74841058a21b..74bb46bd217a 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java @@ -17,37 +17,19 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; - -public class JavaGBTClassifierSuite implements Serializable { - - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaGBTClassifierSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaGBTClassifierSuite extends SharedSparkSession { @Test public void runDT() { @@ -55,7 +37,7 @@ public void runDT() { double A = 2.0; double B = -1.5; - JavaRDD data = sc.parallelize( + JavaRDD data = jsc.parallelize( LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); Map categoricalFeatures = new HashMap<>(); Dataset dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); @@ -74,7 +56,7 @@ public void runDT() { .setMaxIter(3) .setStepSize(0.1) .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern - for (String lossType: GBTClassifier.supportedLossTypes()) { + for (String lossType : GBTClassifier.supportedLossTypes()) { rf.setLossType(lossType); } GBTClassificationModel model = rf.fit(dataFrame); diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java index e160a5a47e30..004102103d52 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java @@ -17,47 +17,34 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; +import java.io.IOException; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.regression.LabeledPoint; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +public class JavaLogisticRegressionSuite extends SharedSparkSession { -public class JavaLogisticRegressionSuite implements Serializable { - - private transient JavaSparkContext jsc; - private transient SQLContext jsql; private transient Dataset dataset; private transient JavaRDD datasetRDD; private double eps = 1e-5; - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaLogisticRegressionSuite"); - jsql = new SQLContext(jsc); + @Override + public void setUp() throws IOException { + super.setUp(); List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); - dataset = jsql.createDataFrame(datasetRDD, LabeledPoint.class); - dataset.registerTempTable("dataset"); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; + dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); + dataset.createOrReplaceTempView("dataset"); } @Test @@ -65,8 +52,8 @@ public void logisticRegressionDefaultParams() { LogisticRegression lr = new LogisticRegression(); Assert.assertEquals(lr.getLabelCol(), "label"); LogisticRegressionModel model = lr.fit(dataset); - model.transform(dataset).registerTempTable("prediction"); - Dataset predictions = jsql.sql("SELECT label, probability, prediction FROM prediction"); + model.transform(dataset).createOrReplaceTempView("prediction"); + Dataset predictions = spark.sql("SELECT label, probability, prediction FROM prediction"); predictions.collectAsList(); // Check defaults Assert.assertEquals(0.5, model.getThreshold(), eps); @@ -94,24 +81,24 @@ public void logisticRegressionWithSetters() { // Modify model params, and check that the params worked. model.setThreshold(1.0); - model.transform(dataset).registerTempTable("predAllZero"); - Dataset predAllZero = jsql.sql("SELECT prediction, myProbability FROM predAllZero"); - for (Row r: predAllZero.collectAsList()) { + model.transform(dataset).createOrReplaceTempView("predAllZero"); + Dataset predAllZero = spark.sql("SELECT prediction, myProbability FROM predAllZero"); + for (Row r : predAllZero.collectAsList()) { Assert.assertEquals(0.0, r.getDouble(0), eps); } // Call transform with params, and check that the params worked. model.transform(dataset, model.threshold().w(0.0), model.probabilityCol().w("myProb")) - .registerTempTable("predNotAllZero"); - Dataset predNotAllZero = jsql.sql("SELECT prediction, myProb FROM predNotAllZero"); + .createOrReplaceTempView("predNotAllZero"); + Dataset predNotAllZero = spark.sql("SELECT prediction, myProb FROM predNotAllZero"); boolean foundNonZero = false; - for (Row r: predNotAllZero.collectAsList()) { + for (Row r : predNotAllZero.collectAsList()) { if (r.getDouble(0) != 0.0) foundNonZero = true; } Assert.assertTrue(foundNonZero); // Call fit() with new params, and check as many params as we can. LogisticRegressionModel model2 = lr.fit(dataset, lr.maxIter().w(5), lr.regParam().w(0.1), - lr.threshold().w(0.4), lr.probabilityCol().w("theProb")); + lr.threshold().w(0.4), lr.probabilityCol().w("theProb")); LogisticRegression parent2 = (LogisticRegression) model2.parent(); Assert.assertEquals(5, parent2.getMaxIter()); Assert.assertEquals(0.1, parent2.getRegParam(), eps); @@ -127,11 +114,11 @@ public void logisticRegressionPredictorClassifierMethods() { LogisticRegressionModel model = lr.fit(dataset); Assert.assertEquals(2, model.numClasses()); - model.transform(dataset).registerTempTable("transformed"); - Dataset trans1 = jsql.sql("SELECT rawPrediction, probability FROM transformed"); - for (Row row: trans1.collectAsList()) { - Vector raw = (Vector)row.get(0); - Vector prob = (Vector)row.get(1); + model.transform(dataset).createOrReplaceTempView("transformed"); + Dataset trans1 = spark.sql("SELECT rawPrediction, probability FROM transformed"); + for (Row row : trans1.collectAsList()) { + Vector raw = (Vector) row.get(0); + Vector prob = (Vector) row.get(1); Assert.assertEquals(raw.size(), 2); Assert.assertEquals(prob.size(), 2); double probFromRaw1 = 1.0 / (1.0 + Math.exp(-raw.apply(1))); @@ -139,11 +126,11 @@ public void logisticRegressionPredictorClassifierMethods() { Assert.assertEquals(0, Math.abs(prob.apply(0) - (1.0 - probFromRaw1)), eps); } - Dataset trans2 = jsql.sql("SELECT prediction, probability FROM transformed"); - for (Row row: trans2.collectAsList()) { + Dataset trans2 = spark.sql("SELECT prediction, probability FROM transformed"); + for (Row row : trans2.collectAsList()) { double pred = row.getDouble(0); - Vector prob = (Vector)row.get(1); - double probOfPred = prob.apply((int)pred); + Vector prob = (Vector) row.get(1); + double probOfPred = prob.apply((int) pred); for (int i = 0; i < prob.size(); ++i) { Assert.assertTrue(probOfPred >= prob.apply(i)); } diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java index bc955f3cf6b0..6d0604d8f9a5 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java @@ -17,58 +17,39 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; -public class JavaMultilayerPerceptronClassifierSuite implements Serializable { - - private transient JavaSparkContext jsc; - private transient SQLContext sqlContext; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaLogisticRegressionSuite"); - sqlContext = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - sqlContext = null; - } +public class JavaMultilayerPerceptronClassifierSuite extends SharedSparkSession { @Test public void testMLPC() { - Dataset dataFrame = sqlContext.createDataFrame( - jsc.parallelize(Arrays.asList( - new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)), - new LabeledPoint(1.0, Vectors.dense(0.0, 1.0)), - new LabeledPoint(1.0, Vectors.dense(1.0, 0.0)), - new LabeledPoint(0.0, Vectors.dense(1.0, 1.0)))), - LabeledPoint.class); + List data = Arrays.asList( + new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)), + new LabeledPoint(1.0, Vectors.dense(0.0, 1.0)), + new LabeledPoint(1.0, Vectors.dense(1.0, 0.0)), + new LabeledPoint(0.0, Vectors.dense(1.0, 1.0)) + ); + Dataset dataFrame = spark.createDataFrame(data, LabeledPoint.class); + MultilayerPerceptronClassifier mlpc = new MultilayerPerceptronClassifier() - .setLayers(new int[] {2, 5, 2}) + .setLayers(new int[]{2, 5, 2}) .setBlockSize(1) .setSeed(123L) .setMaxIter(100); MultilayerPerceptronClassificationModel model = mlpc.fit(dataFrame); Dataset result = model.transform(dataFrame); List predictionAndLabels = result.select("prediction", "label").collectAsList(); - for (Row r: predictionAndLabels) { + for (Row r : predictionAndLabels) { Assert.assertEquals((int) r.getDouble(0), (int) r.getDouble(1)); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java index 45101f286c6d..c2a9e7b58b47 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java @@ -17,43 +17,24 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaNaiveBayesSuite implements Serializable { - - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaLogisticRegressionSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaNaiveBayesSuite extends SharedSparkSession { public void validatePrediction(Dataset predictionAndLabels) { for (Row r : predictionAndLabels.collectAsList()) { @@ -88,7 +69,7 @@ public void testNaiveBayes() { new StructField("features", new VectorUDT(), false, Metadata.empty()) }); - Dataset dataset = jsql.createDataFrame(data, schema); + Dataset dataset = spark.createDataFrame(data, schema); NaiveBayes nb = new NaiveBayes().setSmoothing(0.5).setModelType("multinomial"); NaiveBayesModel model = nb.fit(dataset); diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java index 00f4476841af..6194167bda35 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java @@ -17,69 +17,57 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; +import java.io.IOException; import java.util.List; -import org.apache.spark.sql.Row; import scala.collection.JavaConverters; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateMultinomialLogisticInput; -import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; - -public class JavaOneVsRestSuite implements Serializable { +import org.apache.spark.sql.Row; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateMultinomialLogisticInput; - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - private transient Dataset dataset; - private transient JavaRDD datasetRDD; +public class JavaOneVsRestSuite extends SharedSparkSession { - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaLOneVsRestSuite"); - jsql = new SQLContext(jsc); - int nPoints = 3; + private transient Dataset dataset; + private transient JavaRDD datasetRDD; - // The following coefficients and xMean/xVariance are computed from iris dataset with - // lambda=0.2. - // As a result, we are drawing samples from probability distribution of an actual model. - double[] coefficients = { - -0.57997, 0.912083, -0.371077, -0.819866, 2.688191, - -0.16624, -0.84355, -0.048509, -0.301789, 4.170682 }; + @Override + public void setUp() throws IOException { + super.setUp(); + int nPoints = 3; - double[] xMean = {5.843, 3.057, 3.758, 1.199}; - double[] xVariance = {0.6856, 0.1899, 3.116, 0.581}; - List points = JavaConverters.seqAsJavaListConverter( - generateMultinomialLogisticInput(coefficients, xMean, xVariance, true, nPoints, 42) - ).asJava(); - datasetRDD = jsc.parallelize(points, 2); - dataset = jsql.createDataFrame(datasetRDD, LabeledPoint.class); - } + // The following coefficients and xMean/xVariance are computed from iris dataset with + // lambda=0.2. + // As a result, we are drawing samples from probability distribution of an actual model. + double[] coefficients = { + -0.57997, 0.912083, -0.371077, -0.819866, 2.688191, + -0.16624, -0.84355, -0.048509, -0.301789, 4.170682}; - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } + double[] xMean = {5.843, 3.057, 3.758, 1.199}; + double[] xVariance = {0.6856, 0.1899, 3.116, 0.581}; + List points = JavaConverters.seqAsJavaListConverter( + generateMultinomialLogisticInput(coefficients, xMean, xVariance, true, nPoints, 42) + ).asJava(); + datasetRDD = jsc.parallelize(points, 2); + dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); + } - @Test - public void oneVsRestDefaultParams() { - OneVsRest ova = new OneVsRest(); - ova.setClassifier(new LogisticRegression()); - Assert.assertEquals(ova.getLabelCol() , "label"); - Assert.assertEquals(ova.getPredictionCol() , "prediction"); - OneVsRestModel ovaModel = ova.fit(dataset); - Dataset predictions = ovaModel.transform(dataset).select("label", "prediction"); - predictions.collectAsList(); - Assert.assertEquals(ovaModel.getLabelCol(), "label"); - Assert.assertEquals(ovaModel.getPredictionCol() , "prediction"); - } + @Test + public void oneVsRestDefaultParams() { + OneVsRest ova = new OneVsRest(); + ova.setClassifier(new LogisticRegression()); + Assert.assertEquals(ova.getLabelCol(), "label"); + Assert.assertEquals(ova.getPredictionCol(), "prediction"); + OneVsRestModel ovaModel = ova.fit(dataset); + Dataset predictions = ovaModel.transform(dataset).select("label", "prediction"); + predictions.collectAsList(); + Assert.assertEquals(ovaModel.getLabelCol(), "label"); + Assert.assertEquals(ovaModel.getPredictionCol(), "prediction"); + } } diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java index 4f40fd65b9f1..dd98513f37ec 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java @@ -17,39 +17,21 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; - -public class JavaRandomForestClassifierSuite implements Serializable { - - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaRandomForestClassifierSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaRandomForestClassifierSuite extends SharedSparkSession { @Test public void runDT() { @@ -57,7 +39,7 @@ public void runDT() { double A = 2.0; double B = -1.5; - JavaRDD data = sc.parallelize( + JavaRDD data = jsc.parallelize( LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); Map categoricalFeatures = new HashMap<>(); Dataset dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); @@ -75,22 +57,22 @@ public void runDT() { .setSeed(1234) .setNumTrees(3) .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern - for (String impurity: RandomForestClassifier.supportedImpurities()) { + for (String impurity : RandomForestClassifier.supportedImpurities()) { rf.setImpurity(impurity); } - for (String featureSubsetStrategy: RandomForestClassifier.supportedFeatureSubsetStrategies()) { + for (String featureSubsetStrategy : RandomForestClassifier.supportedFeatureSubsetStrategies()) { rf.setFeatureSubsetStrategy(featureSubsetStrategy); } String[] realStrategies = {".1", ".10", "0.10", "0.1", "0.9", "1.0"}; - for (String strategy: realStrategies) { + for (String strategy : realStrategies) { rf.setFeatureSubsetStrategy(strategy); } String[] integerStrategies = {"1", "10", "100", "1000", "10000"}; - for (String strategy: integerStrategies) { + for (String strategy : integerStrategies) { rf.setFeatureSubsetStrategy(strategy); } String[] invalidStrategies = {"-.1", "-.10", "-0.10", ".0", "0.0", "1.1", "0"}; - for (String strategy: invalidStrategies) { + for (String strategy : invalidStrategies) { try { rf.setFeatureSubsetStrategy(strategy); Assert.fail("Expected exception to be thrown for invalid strategies"); diff --git a/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java index a3fcdb54ee7a..1be6f96f4c94 100644 --- a/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java @@ -17,41 +17,28 @@ package org.apache.spark.ml.clustering; -import java.io.Serializable; +import java.io.IOException; import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; -public class JavaKMeansSuite implements Serializable { +public class JavaKMeansSuite extends SharedSparkSession { private transient int k = 5; - private transient JavaSparkContext sc; private transient Dataset dataset; - private transient SQLContext sql; - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaKMeansSuite"); - sql = new SQLContext(sc); - - dataset = KMeansSuite.generateKMeansData(sql, 50, 3, k); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; + @Override + public void setUp() throws IOException { + super.setUp(); + dataset = KMeansSuite.generateKMeansData(spark, 50, 3, k); } @Test @@ -65,7 +52,7 @@ public void fitAndTransform() { Dataset transformed = model.transform(dataset); List columns = Arrays.asList(transformed.columns()); List expectedColumns = Arrays.asList("features", "prediction"); - for (String column: expectedColumns) { + for (String column : expectedColumns) { assertTrue(columns.contains(column)); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java index 77e3a489a93a..87639380bdcf 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java @@ -20,45 +20,28 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaBucketizerSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaBucketizerSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaBucketizerSuite extends SharedSparkSession { @Test public void bucketizerTest() { double[] splits = {-0.5, 0.0, 0.5}; - StructType schema = new StructType(new StructField[] { + StructType schema = new StructType(new StructField[]{ new StructField("feature", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset dataset = jsql.createDataFrame( + Dataset dataset = spark.createDataFrame( Arrays.asList( RowFactory.create(-0.5), RowFactory.create(-0.3), diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java index ed1ad4c3a316..b7956b6fd3e9 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java @@ -21,43 +21,27 @@ import java.util.List; import edu.emory.mathcs.jtransforms.dct.DoubleDCT_1D; -import org.junit.After; + import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaDCTSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaDCTSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaDCTSuite extends SharedSparkSession { @Test public void javaCompatibilityTest() { - double[] input = new double[] {1D, 2D, 3D, 4D}; - Dataset dataset = jsql.createDataFrame( + double[] input = new double[]{1D, 2D, 3D, 4D}; + Dataset dataset = spark.createDataFrame( Arrays.asList(RowFactory.create(Vectors.dense(input))), new StructType(new StructField[]{ new StructField("vec", (new VectorUDT()), false, Metadata.empty()) diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java index 6e2cc7e8877c..57696d0150a8 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java @@ -20,38 +20,21 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaHashingTFSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaHashingTFSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaHashingTFSuite extends SharedSparkSession { @Test public void hashingTF() { @@ -65,7 +48,7 @@ public void hashingTF() { new StructField("sentence", DataTypes.StringType, false, Metadata.empty()) }); - Dataset sentenceData = jsql.createDataFrame(data, schema); + Dataset sentenceData = spark.createDataFrame(data, schema); Tokenizer tokenizer = new Tokenizer() .setInputCol("sentence") .setOutputCol("words"); diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java index 5bbd9634b2c2..6f877b566875 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java @@ -19,32 +19,15 @@ import java.util.Arrays; -import org.junit.After; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; -public class JavaNormalizerSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaNormalizerSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaNormalizerSuite extends SharedSparkSession { @Test public void normalizer() { @@ -54,7 +37,7 @@ public void normalizer() { new VectorIndexerSuite.FeatureData(Vectors.dense(1.0, 3.0)), new VectorIndexerSuite.FeatureData(Vectors.dense(1.0, 4.0)) )); - Dataset dataFrame = jsql.createDataFrame(points, VectorIndexerSuite.FeatureData.class); + Dataset dataFrame = spark.createDataFrame(points, VectorIndexerSuite.FeatureData.class); Normalizer normalizer = new Normalizer() .setInputCol("features") .setOutputCol("normFeatures"); diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java index 1389d17e7e07..ac479c08418c 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java @@ -23,37 +23,20 @@ import scala.Tuple2; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.function.Function; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.distributed.RowMatrix; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; - -public class JavaPCASuite implements Serializable { - private transient JavaSparkContext jsc; - private transient SQLContext sqlContext; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaPCASuite"); - sqlContext = new SQLContext(jsc); - } - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaPCASuite extends SharedSparkSession { public static class VectorPair implements Serializable { private Vector features = Vectors.dense(0.0); @@ -85,9 +68,25 @@ public void testPCA() { ); JavaRDD dataRDD = jsc.parallelize(points, 2); - RowMatrix mat = new RowMatrix(dataRDD.rdd()); + RowMatrix mat = new RowMatrix(dataRDD.map( + new Function() { + public org.apache.spark.mllib.linalg.Vector call(Vector vector) { + return new org.apache.spark.mllib.linalg.DenseVector(vector.toArray()); + } + } + ).rdd()); + Matrix pc = mat.computePrincipalComponents(3); - JavaRDD expected = mat.multiply(pc).rows().toJavaRDD(); + + mat.multiply(pc).rows().toJavaRDD(); + + JavaRDD expected = mat.multiply(pc).rows().toJavaRDD().map( + new Function() { + public Vector call(org.apache.spark.mllib.linalg.Vector vector) { + return vector.asML(); + } + } + ); JavaRDD featuresExpected = dataRDD.zip(expected).map( new Function, VectorPair>() { @@ -100,7 +99,7 @@ public VectorPair call(Tuple2 pair) { } ); - Dataset df = sqlContext.createDataFrame(featuresExpected, VectorPair.class); + Dataset df = spark.createDataFrame(featuresExpected, VectorPair.class); PCAModel pca = new PCA() .setInputCol("features") .setOutputCol("pca_features") diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java index 6a8bb6480174..df5d34fbe94e 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java @@ -20,38 +20,21 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaPolynomialExpansionSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaPolynomialExpansionSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaPolynomialExpansionSuite extends SharedSparkSession { @Test public void polynomialExpansionTest() { @@ -72,20 +55,20 @@ public void polynomialExpansionTest() { ) ); - StructType schema = new StructType(new StructField[] { + StructType schema = new StructType(new StructField[]{ new StructField("features", new VectorUDT(), false, Metadata.empty()), new StructField("expected", new VectorUDT(), false, Metadata.empty()) }); - Dataset dataset = jsql.createDataFrame(data, schema); + Dataset dataset = spark.createDataFrame(data, schema); List pairs = polyExpansion.transform(dataset) .select("polyFeatures", "expected") .collectAsList(); for (Row r : pairs) { - double[] polyFeatures = ((Vector)r.get(0)).toArray(); - double[] expected = ((Vector)r.get(1)).toArray(); + double[] polyFeatures = ((Vector) r.get(0)).toArray(); + double[] expected = ((Vector) r.get(1)).toArray(); Assert.assertArrayEquals(polyFeatures, expected, 1e-1); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java index 3f6fc333e4e1..dbc0b1db5c00 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java @@ -20,31 +20,14 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; -public class JavaStandardScalerSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaStandardScalerSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaStandardScalerSuite extends SharedSparkSession { @Test public void standardScaler() { @@ -54,7 +37,7 @@ public void standardScaler() { new VectorIndexerSuite.FeatureData(Vectors.dense(1.0, 3.0)), new VectorIndexerSuite.FeatureData(Vectors.dense(1.0, 4.0)) ); - Dataset dataFrame = jsql.createDataFrame(jsc.parallelize(points, 2), + Dataset dataFrame = spark.createDataFrame(jsc.parallelize(points, 2), VectorIndexerSuite.FeatureData.class); StandardScaler scaler = new StandardScaler() .setInputCol("features") diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java index bdcbde5e2622..6480b57e1f79 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java @@ -20,37 +20,19 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaStopWordsRemoverSuite { - - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaStopWordsRemoverSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaStopWordsRemoverSuite extends SharedSparkSession { @Test public void javaCompatibilityTest() { @@ -62,11 +44,11 @@ public void javaCompatibilityTest() { RowFactory.create(Arrays.asList("I", "saw", "the", "red", "baloon")), RowFactory.create(Arrays.asList("Mary", "had", "a", "little", "lamb")) ); - StructType schema = new StructType(new StructField[] { + StructType schema = new StructType(new StructField[]{ new StructField("raw", DataTypes.createArrayType(DataTypes.StringType), false, - Metadata.empty()) + Metadata.empty()) }); - Dataset dataset = jsql.createDataFrame(data, schema); + Dataset dataset = spark.createDataFrame(data, schema); remover.transform(dataset).collect(); } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java index 431779cd2e72..c1928a26b609 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java @@ -20,45 +20,29 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; +import static org.apache.spark.sql.types.DataTypes.*; + import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import static org.apache.spark.sql.types.DataTypes.*; - -public class JavaStringIndexerSuite { - private transient JavaSparkContext jsc; - private transient SQLContext sqlContext; - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaStringIndexerSuite"); - sqlContext = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - sqlContext = null; - } +public class JavaStringIndexerSuite extends SharedSparkSession { @Test public void testStringIndexer() { - StructType schema = createStructType(new StructField[] { + StructType schema = createStructType(new StructField[]{ createStructField("id", IntegerType, false), createStructField("label", StringType, false) }); List data = Arrays.asList( cr(0, "a"), cr(1, "b"), cr(2, "c"), cr(3, "a"), cr(4, "a"), cr(5, "c")); - Dataset dataset = sqlContext.createDataFrame(data, schema); + Dataset dataset = spark.createDataFrame(data, schema); StringIndexer indexer = new StringIndexer() .setInputCol("label") @@ -70,7 +54,9 @@ public void testStringIndexer() { output.orderBy("id").select("id", "labelIndex").collectAsList()); } - /** An alias for RowFactory.create. */ + /** + * An alias for RowFactory.create. + */ private Row cr(Object... values) { return RowFactory.create(values); } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java index 83d16cbd0e7a..27550a3d5c37 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java @@ -20,32 +20,15 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; -public class JavaTokenizerSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaTokenizerSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaTokenizerSuite extends SharedSparkSession { @Test public void regexTokenizer() { @@ -59,10 +42,10 @@ public void regexTokenizer() { JavaRDD rdd = jsc.parallelize(Arrays.asList( - new TokenizerTestData("Test of tok.", new String[] {"Test", "tok."}), - new TokenizerTestData("Te,st. punct", new String[] {"Te,st.", "punct"}) + new TokenizerTestData("Test of tok.", new String[]{"Test", "tok."}), + new TokenizerTestData("Te,st. punct", new String[]{"Te,st.", "punct"}) )); - Dataset dataset = jsql.createDataFrame(rdd, TokenizerTestData.class); + Dataset dataset = spark.createDataFrame(rdd, TokenizerTestData.class); List pairs = myRegExTokenizer.transform(dataset) .select("tokens", "wantedTokens") diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java index e45e19804345..583652badb8f 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java @@ -19,41 +19,26 @@ import java.util.Arrays; -import org.junit.After; +import static org.apache.spark.sql.types.DataTypes.*; + import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.types.*; -import static org.apache.spark.sql.types.DataTypes.*; - -public class JavaVectorAssemblerSuite { - private transient JavaSparkContext jsc; - private transient SQLContext sqlContext; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaVectorAssemblerSuite"); - sqlContext = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaVectorAssemblerSuite extends SharedSparkSession { @Test public void testVectorAssembler() { - StructType schema = createStructType(new StructField[] { + StructType schema = createStructType(new StructField[]{ createStructField("id", IntegerType, false), createStructField("x", DoubleType, false), createStructField("y", new VectorUDT(), false), @@ -63,14 +48,14 @@ public void testVectorAssembler() { }); Row row = RowFactory.create( 0, 0.0, Vectors.dense(1.0, 2.0), "a", - Vectors.sparse(2, new int[] {1}, new double[] {3.0}), 10L); - Dataset dataset = sqlContext.createDataFrame(Arrays.asList(row), schema); + Vectors.sparse(2, new int[]{1}, new double[]{3.0}), 10L); + Dataset dataset = spark.createDataFrame(Arrays.asList(row), schema); VectorAssembler assembler = new VectorAssembler() - .setInputCols(new String[] {"x", "y", "z", "n"}) + .setInputCols(new String[]{"x", "y", "z", "n"}) .setOutputCol("features"); Dataset output = assembler.transform(dataset); Assert.assertEquals( - Vectors.sparse(6, new int[] {1, 2, 4, 5}, new double[] {1.0, 2.0, 3.0, 10.0}), + Vectors.sparse(6, new int[]{1, 2, 4, 5}, new double[]{1.0, 2.0, 3.0, 10.0}), output.select("features").first().getAs(0)); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java index fec6cac8bec3..ca8fae3a48b9 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java @@ -17,37 +17,21 @@ package org.apache.spark.ml.feature; -import java.io.Serializable; import java.util.Arrays; import java.util.List; import java.util.Map; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.feature.VectorIndexerSuite.FeatureData; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; -public class JavaVectorIndexerSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaVectorIndexerSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaVectorIndexerSuite extends SharedSparkSession { @Test public void vectorIndexerAPI() { @@ -57,8 +41,7 @@ public void vectorIndexerAPI() { new FeatureData(Vectors.dense(1.0, 3.0)), new FeatureData(Vectors.dense(1.0, 4.0)) ); - SQLContext sqlContext = new SQLContext(sc); - Dataset data = sqlContext.createDataFrame(sc.parallelize(points, 2), FeatureData.class); + Dataset data = spark.createDataFrame(jsc.parallelize(points, 2), FeatureData.class); VectorIndexer indexer = new VectorIndexer() .setInputCol("features") .setOutputCol("indexed") diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java index e2da11183b93..3dc2e1f89614 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java @@ -20,39 +20,22 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.attribute.Attribute; import org.apache.spark.ml.attribute.AttributeGroup; import org.apache.spark.ml.attribute.NumericAttribute; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.StructType; -public class JavaVectorSlicerSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaVectorSlicerSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaVectorSlicerSuite extends SharedSparkSession { @Test public void vectorSlice() { @@ -69,7 +52,7 @@ public void vectorSlice() { ); Dataset dataset = - jsql.createDataFrame(data, (new StructType()).add(group.toStructField())); + spark.createDataFrame(data, (new StructType()).add(group.toStructField())); VectorSlicer vectorSlicer = new VectorSlicer() .setInputCol("userFeatures").setOutputCol("features"); diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java index 7517b70cc9be..d0a849fd11c7 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java @@ -19,41 +19,24 @@ import java.util.Arrays; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.*; -public class JavaWord2VecSuite { - private transient JavaSparkContext jsc; - private transient SQLContext sqlContext; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaWord2VecSuite"); - sqlContext = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaWord2VecSuite extends SharedSparkSession { @Test public void testJavaWord2Vec() { StructType schema = new StructType(new StructField[]{ new StructField("text", new ArrayType(DataTypes.StringType, true), false, Metadata.empty()) }); - Dataset documentDF = sqlContext.createDataFrame( + Dataset documentDF = spark.createDataFrame( Arrays.asList( RowFactory.create(Arrays.asList("Hi I heard about Spark".split(" "))), RowFactory.create(Arrays.asList("I wish Java could use case classes".split(" "))), @@ -68,8 +51,8 @@ public void testJavaWord2Vec() { Word2VecModel model = word2Vec.fit(documentDF); Dataset result = model.transform(documentDF); - for (Row r: result.select("result").collectAsList()) { - double[] polyFeatures = ((Vector)r.get(0)).toArray(); + for (Row r : result.select("result").collectAsList()) { + double[] polyFeatures = ((Vector) r.get(0)).toArray(); Assert.assertEquals(polyFeatures.length, 3); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java b/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java index fa777f3d42a9..1077e103a3b8 100644 --- a/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java @@ -19,31 +19,14 @@ import java.util.Arrays; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; - /** * Test Param and related classes in Java */ public class JavaParamsSuite { - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaParamsSuite"); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } - @Test public void testParams() { JavaTestParams testParams = new JavaTestParams(); @@ -51,7 +34,7 @@ public void testParams() { testParams.setMyIntParam(2).setMyDoubleParam(0.4).setMyStringParam("a"); Assert.assertEquals(testParams.getMyDoubleParam(), 0.4, 0.0); Assert.assertEquals(testParams.getMyStringParam(), "a"); - Assert.assertArrayEquals(testParams.getMyDoubleArrayParam(), new double[] {1.0, 2.0}, 0.0); + Assert.assertArrayEquals(testParams.getMyDoubleArrayParam(), new double[]{1.0, 2.0}, 0.0); } @Test diff --git a/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java b/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java index 06f7fbb86e88..1ad5f7a442da 100644 --- a/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java +++ b/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java @@ -45,9 +45,14 @@ public String uid() { } private IntParam myIntParam_; - public IntParam myIntParam() { return myIntParam_; } - public int getMyIntParam() { return (Integer)getOrDefault(myIntParam_); } + public IntParam myIntParam() { + return myIntParam_; + } + + public int getMyIntParam() { + return (Integer) getOrDefault(myIntParam_); + } public JavaTestParams setMyIntParam(int value) { set(myIntParam_, value); @@ -55,9 +60,14 @@ public JavaTestParams setMyIntParam(int value) { } private DoubleParam myDoubleParam_; - public DoubleParam myDoubleParam() { return myDoubleParam_; } - public double getMyDoubleParam() { return (Double)getOrDefault(myDoubleParam_); } + public DoubleParam myDoubleParam() { + return myDoubleParam_; + } + + public double getMyDoubleParam() { + return (Double) getOrDefault(myDoubleParam_); + } public JavaTestParams setMyDoubleParam(double value) { set(myDoubleParam_, value); @@ -65,9 +75,14 @@ public JavaTestParams setMyDoubleParam(double value) { } private Param myStringParam_; - public Param myStringParam() { return myStringParam_; } - public String getMyStringParam() { return getOrDefault(myStringParam_); } + public Param myStringParam() { + return myStringParam_; + } + + public String getMyStringParam() { + return getOrDefault(myStringParam_); + } public JavaTestParams setMyStringParam(String value) { set(myStringParam_, value); @@ -75,9 +90,14 @@ public JavaTestParams setMyStringParam(String value) { } private DoubleArrayParam myDoubleArrayParam_; - public DoubleArrayParam myDoubleArrayParam() { return myDoubleArrayParam_; } - public double[] getMyDoubleArrayParam() { return getOrDefault(myDoubleArrayParam_); } + public DoubleArrayParam myDoubleArrayParam() { + return myDoubleArrayParam_; + } + + public double[] getMyDoubleArrayParam() { + return getOrDefault(myDoubleArrayParam_); + } public JavaTestParams setMyDoubleArrayParam(double[] value) { set(myDoubleArrayParam_, value); @@ -96,7 +116,7 @@ private void init() { setDefault(myIntParam(), 1); setDefault(myDoubleParam(), 0.5); - setDefault(myDoubleArrayParam(), new double[] {1.0, 2.0}); + setDefault(myDoubleArrayParam(), new double[]{1.0, 2.0}); } @Override diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java index fa3b28ed4f30..1da85ed9dab4 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java @@ -17,37 +17,21 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -public class JavaDecisionTreeRegressorSuite implements Serializable { - - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaDecisionTreeRegressorSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaDecisionTreeRegressorSuite extends SharedSparkSession { @Test public void runDT() { @@ -55,7 +39,7 @@ public void runDT() { double A = 2.0; double B = -1.5; - JavaRDD data = sc.parallelize( + JavaRDD data = jsc.parallelize( LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); Map categoricalFeatures = new HashMap<>(); Dataset dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 0); @@ -70,7 +54,7 @@ public void runDT() { .setCacheNodeIds(false) .setCheckpointInterval(10) .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern - for (String impurity: DecisionTreeRegressor.supportedImpurities()) { + for (String impurity : DecisionTreeRegressor.supportedImpurities()) { dt.setImpurity(impurity); } DecisionTreeRegressionModel model = dt.fit(dataFrame); diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java index 8413ea0e0a94..7fd9b1feb7f8 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java @@ -17,37 +17,21 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -public class JavaGBTRegressorSuite implements Serializable { - - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaGBTRegressorSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaGBTRegressorSuite extends SharedSparkSession { @Test public void runDT() { @@ -55,7 +39,7 @@ public void runDT() { double A = 2.0; double B = -1.5; - JavaRDD data = sc.parallelize( + JavaRDD data = jsc.parallelize( LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); Map categoricalFeatures = new HashMap<>(); Dataset dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 0); @@ -73,7 +57,7 @@ public void runDT() { .setMaxIter(3) .setStepSize(0.1) .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern - for (String lossType: GBTRegressor.supportedLossTypes()) { + for (String lossType : GBTRegressor.supportedLossTypes()) { rf.setLossType(lossType); } GBTRegressionModel model = rf.fit(dataFrame); diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java index 9f817515eb86..6cdcdda1a648 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java @@ -17,45 +17,30 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; +import java.io.IOException; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.regression.LabeledPoint; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite - .generateLogisticInputAsList; - -public class JavaLinearRegressionSuite implements Serializable { - - private transient JavaSparkContext jsc; - private transient SQLContext jsql; +public class JavaLinearRegressionSuite extends SharedSparkSession { private transient Dataset dataset; private transient JavaRDD datasetRDD; - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaLinearRegressionSuite"); - jsql = new SQLContext(jsc); + @Override + public void setUp() throws IOException { + super.setUp(); List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); - dataset = jsql.createDataFrame(datasetRDD, LabeledPoint.class); - dataset.registerTempTable("dataset"); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; + dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); + dataset.createOrReplaceTempView("dataset"); } @Test @@ -64,8 +49,8 @@ public void linearRegressionDefaultParams() { assertEquals("label", lr.getLabelCol()); assertEquals("auto", lr.getSolver()); LinearRegressionModel model = lr.fit(dataset); - model.transform(dataset).registerTempTable("prediction"); - Dataset predictions = jsql.sql("SELECT label, prediction FROM prediction"); + model.transform(dataset).createOrReplaceTempView("prediction"); + Dataset predictions = spark.sql("SELECT label, prediction FROM prediction"); predictions.collect(); // Check defaults assertEquals("features", model.getFeaturesCol()); @@ -76,8 +61,8 @@ public void linearRegressionDefaultParams() { public void linearRegressionWithSetters() { // Set params, train, and check as many params as we can. LinearRegression lr = new LinearRegression() - .setMaxIter(10) - .setRegParam(1.0).setSolver("l-bfgs"); + .setMaxIter(10) + .setRegParam(1.0).setSolver("l-bfgs"); LinearRegressionModel model = lr.fit(dataset); LinearRegression parent = (LinearRegression) model.parent(); assertEquals(10, parent.getMaxIter()); @@ -85,7 +70,7 @@ public void linearRegressionWithSetters() { // Call fit() with new params, and check as many params as we can. LinearRegressionModel model2 = - lr.fit(dataset, lr.maxIter().w(5), lr.regParam().w(0.1), lr.predictionCol().w("thePred")); + lr.fit(dataset, lr.maxIter().w(5), lr.regParam().w(0.1), lr.predictionCol().w("thePred")); LinearRegression parent2 = (LinearRegression) model2.parent(); assertEquals(5, parent2.getMaxIter()); assertEquals(0.1, parent2.getRegParam(), 0.0); diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java index 38b895f1fdd7..4ba13e2e06c8 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java @@ -17,39 +17,23 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -public class JavaRandomForestRegressorSuite implements Serializable { - - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaRandomForestRegressorSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaRandomForestRegressorSuite extends SharedSparkSession { @Test public void runDT() { @@ -57,7 +41,7 @@ public void runDT() { double A = 2.0; double B = -1.5; - JavaRDD data = sc.parallelize( + JavaRDD data = jsc.parallelize( LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); Map categoricalFeatures = new HashMap<>(); Dataset dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 0); @@ -75,22 +59,22 @@ public void runDT() { .setSeed(1234) .setNumTrees(3) .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern - for (String impurity: RandomForestRegressor.supportedImpurities()) { + for (String impurity : RandomForestRegressor.supportedImpurities()) { rf.setImpurity(impurity); } - for (String featureSubsetStrategy: RandomForestRegressor.supportedFeatureSubsetStrategies()) { + for (String featureSubsetStrategy : RandomForestRegressor.supportedFeatureSubsetStrategies()) { rf.setFeatureSubsetStrategy(featureSubsetStrategy); } String[] realStrategies = {".1", ".10", "0.10", "0.1", "0.9", "1.0"}; - for (String strategy: realStrategies) { + for (String strategy : realStrategies) { rf.setFeatureSubsetStrategy(strategy); } String[] integerStrategies = {"1", "10", "100", "1000", "10000"}; - for (String strategy: integerStrategies) { + for (String strategy : integerStrategies) { rf.setFeatureSubsetStrategy(strategy); } String[] invalidStrategies = {"-.1", "-.10", "-0.10", ".0", "0.0", "1.1", "0"}; - for (String strategy: invalidStrategies) { + for (String strategy : invalidStrategies) { try { rf.setFeatureSubsetStrategy(strategy); Assert.fail("Expected exception to be thrown for invalid strategies"); diff --git a/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java b/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java index 1c18b2b266fe..fa39f4560c8a 100644 --- a/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java @@ -23,35 +23,28 @@ import com.google.common.io.Files; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.DenseVector; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.DenseVector; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; import org.apache.spark.util.Utils; /** * Test LibSVMRelation in Java. */ -public class JavaLibSVMRelationSuite { - private transient JavaSparkContext jsc; - private transient SQLContext sqlContext; +public class JavaLibSVMRelationSuite extends SharedSparkSession { private File tempDir; private String path; - @Before + @Override public void setUp() throws IOException { - jsc = new JavaSparkContext("local", "JavaLibSVMRelationSuite"); - sqlContext = new SQLContext(jsc); - + super.setUp(); tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "datasource"); File file = new File(tempDir, "part-00000"); String s = "1 1:1.0 3:2.0 5:3.0\n0\n0 2:4.0 4:5.0 6:6.0"; @@ -59,16 +52,15 @@ public void setUp() throws IOException { path = tempDir.toURI().toString(); } - @After + @Override public void tearDown() { - jsc.stop(); - jsc = null; + super.tearDown(); Utils.deleteRecursively(tempDir); } @Test public void verifyLibSVMDF() { - Dataset dataset = sqlContext.read().format("libsvm").option("vectorType", "dense") + Dataset dataset = spark.read().format("libsvm").option("vectorType", "dense") .load(path); Assert.assertEquals("label", dataset.columns()[0]); Assert.assertEquals("features", dataset.columns()[1]); diff --git a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java index 24b0097454fe..692d5ad591e8 100644 --- a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java @@ -17,50 +17,39 @@ package org.apache.spark.ml.tuning; -import java.io.Serializable; +import java.io.IOException; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.param.ParamMap; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; -public class JavaCrossValidatorSuite implements Serializable { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; +public class JavaCrossValidatorSuite extends SharedSparkSession { + private transient Dataset dataset; - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaCrossValidatorSuite"); - jsql = new SQLContext(jsc); + @Override + public void setUp() throws IOException { + super.setUp(); List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); - dataset = jsql.createDataFrame(jsc.parallelize(points, 2), LabeledPoint.class); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; + dataset = spark.createDataFrame(jsc.parallelize(points, 2), LabeledPoint.class); } @Test public void crossValidationWithLogisticRegression() { LogisticRegression lr = new LogisticRegression(); ParamMap[] lrParamMaps = new ParamGridBuilder() - .addGrid(lr.regParam(), new double[] {0.001, 1000.0}) - .addGrid(lr.maxIter(), new int[] {0, 10}) + .addGrid(lr.regParam(), new double[]{0.001, 1000.0}) + .addGrid(lr.maxIter(), new int[]{0, 10}) .build(); BinaryClassificationEvaluator eval = new BinaryClassificationEvaluator(); CrossValidator cv = new CrossValidator() diff --git a/mllib/src/test/java/org/apache/spark/ml/util/IdentifiableSuite.scala b/mllib/src/test/java/org/apache/spark/ml/util/IdentifiableSuite.scala index 928301523fba..878bc66ee37c 100644 --- a/mllib/src/test/java/org/apache/spark/ml/util/IdentifiableSuite.scala +++ b/mllib/src/test/java/org/apache/spark/ml/util/IdentifiableSuite.scala @@ -37,4 +37,5 @@ object IdentifiableSuite { class Test(override val uid: String) extends Identifiable { def this() = this(Identifiable.randomUID("test")) } + } diff --git a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java index 01ff1ea65861..7bda219243bf 100644 --- a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java @@ -20,39 +20,25 @@ import java.io.File; import java.io.IOException; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.util.Utils; -public class JavaDefaultReadWriteSuite { - - JavaSparkContext jsc = null; - SQLContext sqlContext = null; +public class JavaDefaultReadWriteSuite extends SharedSparkSession { File tempDir = null; - @Before - public void setUp() { - jsc = new JavaSparkContext("local[2]", "JavaDefaultReadWriteSuite"); - SQLContext.clearActive(); - sqlContext = new SQLContext(jsc); - SQLContext.setActive(sqlContext); + @Override + public void setUp() throws IOException { + super.setUp(); tempDir = Utils.createTempDir( System.getProperty("java.io.tmpdir"), "JavaDefaultReadWriteSuite"); } - @After + @Override public void tearDown() { - sqlContext = null; - SQLContext.clearActive(); - if (jsc != null) { - jsc.stop(); - jsc = null; - } + super.tearDown(); Utils.deleteRecursively(tempDir); } @@ -70,7 +56,7 @@ public void testDefaultReadWrite() throws IOException { } catch (IOException e) { // expected } - instance.write().context(sqlContext).overwrite().save(outputPath); + instance.write().context(spark.sqlContext()).overwrite().save(outputPath); MyParams newInstance = MyParams.load(outputPath); Assert.assertEquals("UID should match.", instance.uid(), newInstance.uid()); Assert.assertEquals("Params should be preserved.", diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java index 862221d48798..c04e2e69541b 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java @@ -17,36 +17,20 @@ package org.apache.spark.mllib.classification; -import java.io.Serializable; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; - import org.apache.spark.mllib.regression.LabeledPoint; -public class JavaLogisticRegressionSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaLogisticRegressionSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaLogisticRegressionSuite extends SharedSparkSession { int validatePrediction(List validationData, LogisticRegressionModel model) { int numAccurate = 0; - for (LabeledPoint point: validationData) { + for (LabeledPoint point : validationData) { Double prediction = model.predict(point.features()); if (prediction == point.label()) { numAccurate++; @@ -61,16 +45,16 @@ public void runLRUsingConstructor() { double A = 2.0; double B = -1.5; - JavaRDD testRDD = sc.parallelize( - LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + JavaRDD testRDD = jsc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); List validationData = - LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); LogisticRegressionWithSGD lrImpl = new LogisticRegressionWithSGD(); lrImpl.setIntercept(true); lrImpl.optimizer().setStepSize(1.0) - .setRegParam(1.0) - .setNumIterations(100); + .setRegParam(1.0) + .setNumIterations(100); LogisticRegressionModel model = lrImpl.run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); @@ -83,13 +67,13 @@ public void runLRUsingStaticMethods() { double A = 0.0; double B = -2.5; - JavaRDD testRDD = sc.parallelize( - LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + JavaRDD testRDD = jsc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); List validationData = - LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); LogisticRegressionModel model = LogisticRegressionWithSGD.train( - testRDD.rdd(), 100, 1.0, 1.0); + testRDD.rdd(), 100, 1.0, 1.0); int numAccurate = validatePrediction(validationData, model); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java index 3771c0ea7ad8..6ded42e92825 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java @@ -17,36 +17,21 @@ package org.apache.spark.mllib.classification; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; -public class JavaNaiveBayesSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaNaiveBayesSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaNaiveBayesSuite extends SharedSparkSession { private static final List POINTS = Arrays.asList( new LabeledPoint(0, Vectors.dense(1.0, 0.0, 0.0)), @@ -59,7 +44,7 @@ public void tearDown() { private int validatePrediction(List points, NaiveBayesModel model) { int correct = 0; - for (LabeledPoint p: points) { + for (LabeledPoint p : points) { if (model.predict(p.features()) == p.label()) { correct += 1; } @@ -69,7 +54,7 @@ private int validatePrediction(List points, NaiveBayesModel model) @Test public void runUsingConstructor() { - JavaRDD testRDD = sc.parallelize(POINTS, 2).cache(); + JavaRDD testRDD = jsc.parallelize(POINTS, 2).cache(); NaiveBayes nb = new NaiveBayes().setLambda(1.0); NaiveBayesModel model = nb.run(testRDD.rdd()); @@ -80,7 +65,7 @@ public void runUsingConstructor() { @Test public void runUsingStaticMethods() { - JavaRDD testRDD = sc.parallelize(POINTS, 2).cache(); + JavaRDD testRDD = jsc.parallelize(POINTS, 2).cache(); NaiveBayesModel model1 = NaiveBayes.train(testRDD.rdd()); int numAccurate1 = validatePrediction(POINTS, model1); @@ -93,13 +78,14 @@ public void runUsingStaticMethods() { @Test public void testPredictJavaRDD() { - JavaRDD examples = sc.parallelize(POINTS, 2).cache(); + JavaRDD examples = jsc.parallelize(POINTS, 2).cache(); NaiveBayesModel model = NaiveBayes.train(examples.rdd()); JavaRDD vectors = examples.map(new Function() { @Override public Vector call(LabeledPoint v) throws Exception { return v.features(); - }}); + } + }); JavaRDD predictions = model.predict(vectors); // Should be able to get the first prediction. predictions.first(); diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java index 31b9f3e8d438..0f54e684e447 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java @@ -17,35 +17,20 @@ package org.apache.spark.mllib.classification; -import java.io.Serializable; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.regression.LabeledPoint; -public class JavaSVMSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaSVMSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaSVMSuite extends SharedSparkSession { int validatePrediction(List validationData, SVMModel model) { int numAccurate = 0; - for (LabeledPoint point: validationData) { + for (LabeledPoint point : validationData) { Double prediction = model.predict(point.features()); if (prediction == point.label()) { numAccurate++; @@ -60,16 +45,16 @@ public void runSVMUsingConstructor() { double A = 2.0; double[] weights = {-1.5, 1.0}; - JavaRDD testRDD = sc.parallelize(SVMSuite.generateSVMInputAsList(A, - weights, nPoints, 42), 2).cache(); + JavaRDD testRDD = jsc.parallelize(SVMSuite.generateSVMInputAsList(A, + weights, nPoints, 42), 2).cache(); List validationData = - SVMSuite.generateSVMInputAsList(A, weights, nPoints, 17); + SVMSuite.generateSVMInputAsList(A, weights, nPoints, 17); SVMWithSGD svmSGDImpl = new SVMWithSGD(); svmSGDImpl.setIntercept(true); svmSGDImpl.optimizer().setStepSize(1.0) - .setRegParam(1.0) - .setNumIterations(100); + .setRegParam(1.0) + .setNumIterations(100); SVMModel model = svmSGDImpl.run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); @@ -82,10 +67,10 @@ public void runSVMUsingStaticMethods() { double A = 0.0; double[] weights = {-1.5, 1.0}; - JavaRDD testRDD = sc.parallelize(SVMSuite.generateSVMInputAsList(A, - weights, nPoints, 42), 2).cache(); + JavaRDD testRDD = jsc.parallelize(SVMSuite.generateSVMInputAsList(A, + weights, nPoints, 42), 2).cache(); List validationData = - SVMSuite.generateSVMInputAsList(A, weights, nPoints, 17); + SVMSuite.generateSVMInputAsList(A, weights, nPoints, 17); SVMModel model = SVMWithSGD.train(testRDD.rdd(), 100, 1.0, 1.0, 1.0); diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java index 62c6d9b7e390..8c6bced52dd7 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.classification; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -37,7 +36,7 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext; import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStreamingLogisticRegressionSuite implements Serializable { +public class JavaStreamingLogisticRegressionSuite { protected transient JavaStreamingContext ssc; diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java index a714620ff7e4..3d62b273d221 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java @@ -17,39 +17,24 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; - import com.google.common.collect.Lists; -import org.junit.After; + import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; -public class JavaBisectingKMeansSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", this.getClass().getSimpleName()); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaBisectingKMeansSuite extends SharedSparkSession { @Test public void twoDimensionalData() { - JavaRDD points = sc.parallelize(Lists.newArrayList( + JavaRDD points = jsc.parallelize(Lists.newArrayList( Vectors.dense(4, -1), Vectors.dense(4, 1), - Vectors.sparse(2, new int[] {0}, new double[] {1.0}) + Vectors.sparse(2, new int[]{0}, new double[]{1.0}) ), 2); BisectingKMeans bkm = new BisectingKMeans() @@ -58,15 +43,15 @@ public void twoDimensionalData() { .setSeed(1L); BisectingKMeansModel model = bkm.run(points); Assert.assertEquals(3, model.k()); - Assert.assertArrayEquals(new double[] {3.0, 0.0}, model.root().center().toArray(), 1e-12); - for (ClusteringTreeNode child: model.root().children()) { + Assert.assertArrayEquals(new double[]{3.0, 0.0}, model.root().center().toArray(), 1e-12); + for (ClusteringTreeNode child : model.root().children()) { double[] center = child.center().toArray(); if (center[0] > 2) { Assert.assertEquals(2, child.size()); - Assert.assertArrayEquals(new double[] {4.0, 0.0}, center, 1e-12); + Assert.assertArrayEquals(new double[]{4.0, 0.0}, center, 1e-12); } else { Assert.assertEquals(1, child.size()); - Assert.assertArrayEquals(new double[] {1.0, 0.0}, center, 1e-12); + Assert.assertArrayEquals(new double[]{1.0, 0.0}, center, 1e-12); } } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java index 123f78da54e3..bf7671993777 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java @@ -17,34 +17,19 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - import static org.junit.Assert.assertEquals; +import org.junit.Test; + +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; -public class JavaGaussianMixtureSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaGaussianMixture"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaGaussianMixtureSuite extends SharedSparkSession { @Test public void runGaussianMixture() { @@ -54,7 +39,7 @@ public void runGaussianMixture() { Vectors.dense(1.0, 4.0, 6.0) ); - JavaRDD data = sc.parallelize(points, 2); + JavaRDD data = jsc.parallelize(points, 2); GaussianMixtureModel model = new GaussianMixture().setK(2).setMaxIterations(1).setSeed(1234) .run(data); assertEquals(model.gaussians().length, 2); diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java index ad06676c72ac..270e636f8211 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java @@ -17,33 +17,19 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; +import static org.junit.Assert.assertEquals; + import org.junit.Test; -import static org.junit.Assert.*; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; -public class JavaKMeansSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaKMeans"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaKMeansSuite extends SharedSparkSession { @Test public void runKMeansUsingStaticMethods() { @@ -55,7 +41,7 @@ public void runKMeansUsingStaticMethods() { Vector expectedCenter = Vectors.dense(1.0, 3.0, 4.0); - JavaRDD data = sc.parallelize(points, 2); + JavaRDD data = jsc.parallelize(points, 2); KMeansModel model = KMeans.train(data.rdd(), 1, 1, 1, KMeans.K_MEANS_PARALLEL()); assertEquals(1, model.clusterCenters().length); assertEquals(expectedCenter, model.clusterCenters()[0]); @@ -74,7 +60,7 @@ public void runKMeansUsingConstructor() { Vector expectedCenter = Vectors.dense(1.0, 3.0, 4.0); - JavaRDD data = sc.parallelize(points, 2); + JavaRDD data = jsc.parallelize(points, 2); KMeansModel model = new KMeans().setK(1).setMaxIterations(5).run(data.rdd()); assertEquals(1, model.clusterCenters().length); assertEquals(expectedCenter, model.clusterCenters()[0]); @@ -94,7 +80,7 @@ public void testPredictJavaRDD() { Vectors.dense(1.0, 3.0, 0.0), Vectors.dense(1.0, 4.0, 6.0) ); - JavaRDD data = sc.parallelize(points, 2); + JavaRDD data = jsc.parallelize(points, 2); KMeansModel model = new KMeans().setK(1).setMaxIterations(5).run(data.rdd()); JavaRDD predictions = model.predict(data); // Should be able to get the first prediction. diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java index db19b309f65a..08d6713ab2bc 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java @@ -17,49 +17,37 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import scala.Tuple2; import scala.Tuple3; -import org.junit.After; -import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; -import org.apache.spark.api.java.function.Function; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Matrix; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; -public class JavaLDASuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaLDA"); +public class JavaLDASuite extends SharedSparkSession { + @Override + public void setUp() throws IOException { + super.setUp(); ArrayList> tinyCorpus = new ArrayList<>(); for (int i = 0; i < LDASuite.tinyCorpus().length; i++) { - tinyCorpus.add(new Tuple2<>((Long)LDASuite.tinyCorpus()[i]._1(), - LDASuite.tinyCorpus()[i]._2())); + tinyCorpus.add(new Tuple2<>((Long) LDASuite.tinyCorpus()[i]._1(), + LDASuite.tinyCorpus()[i]._2())); } - JavaRDD> tmpCorpus = sc.parallelize(tinyCorpus, 2); + JavaRDD> tmpCorpus = jsc.parallelize(tinyCorpus, 2); corpus = JavaPairRDD.fromJavaRDD(tmpCorpus); } - @After - public void tearDown() { - sc.stop(); - sc = null; - } - @Test public void localLDAModel() { Matrix topics = LDASuite.tinyTopics(); @@ -95,7 +83,7 @@ public void distributedLDAModel() { .setMaxIterations(5) .setSeed(12345); - DistributedLDAModel model = (DistributedLDAModel)lda.run(corpus); + DistributedLDAModel model = (DistributedLDAModel) lda.run(corpus); // Check: basic parameters LocalLDAModel localModel = model.toLocal(); @@ -124,7 +112,7 @@ public void distributedLDAModel() { public Boolean call(Tuple2 tuple2) { return Vectors.norm(tuple2._2(), 1.0) != 0.0; } - }); + }); assertEquals(topicDistributions.count(), nonEmptyCorpus.count()); // Check: javaTopTopicsPerDocuments @@ -179,7 +167,7 @@ public void onlineOptimizerCompatibility() { @Test public void localLdaMethods() { - JavaRDD> docs = sc.parallelize(toyData, 2); + JavaRDD> docs = jsc.parallelize(toyData, 2); JavaPairRDD pairedDocs = JavaPairRDD.fromJavaRDD(docs); // check: topicDistributions @@ -191,7 +179,7 @@ public void localLdaMethods() { // check: logLikelihood. ArrayList> docsSingleWord = new ArrayList<>(); docsSingleWord.add(new Tuple2<>(0L, Vectors.dense(1.0, 0.0, 0.0))); - JavaPairRDD single = JavaPairRDD.fromJavaRDD(sc.parallelize(docsSingleWord)); + JavaPairRDD single = JavaPairRDD.fromJavaRDD(jsc.parallelize(docsSingleWord)); double logLikelihood = toyModel.logLikelihood(single); } @@ -199,7 +187,7 @@ public void localLdaMethods() { private static int tinyVocabSize = LDASuite.tinyVocabSize(); private static Matrix tinyTopics = LDASuite.tinyTopics(); private static Tuple2[] tinyTopicDescription = - LDASuite.tinyTopicDescription(); + LDASuite.tinyTopicDescription(); private JavaPairRDD corpus; private LocalLDAModel toyModel = LDASuite.toyModel(); private ArrayList> toyData = LDASuite.javaToyData(); diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java index 62edbd3a298c..d41fc0e4dca9 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -27,8 +26,6 @@ import org.junit.Before; import org.junit.Test; -import static org.apache.spark.streaming.JavaTestUtils.*; - import org.apache.spark.SparkConf; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; @@ -36,8 +33,9 @@ import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; +import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStreamingKMeansSuite implements Serializable { +public class JavaStreamingKMeansSuite { protected transient JavaStreamingContext ssc; diff --git a/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java index fa4d334801ce..e9d7e4fdbe8c 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java @@ -17,41 +17,32 @@ package org.apache.spark.mllib.evaluation; -import java.io.Serializable; +import java.io.IOException; import java.util.Arrays; import java.util.List; import scala.Tuple2; import scala.Tuple2$; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class JavaRankingMetricsSuite implements Serializable { - private transient JavaSparkContext sc; +public class JavaRankingMetricsSuite extends SharedSparkSession { private transient JavaRDD, List>> predictionAndLabels; - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaRankingMetricsSuite"); - predictionAndLabels = sc.parallelize(Arrays.asList( + @Override + public void setUp() throws IOException { + super.setUp(); + predictionAndLabels = jsc.parallelize(Arrays.asList( Tuple2$.MODULE$.apply( Arrays.asList(1, 6, 2, 7, 8, 3, 9, 10, 4, 5), Arrays.asList(1, 2, 3, 4, 5)), Tuple2$.MODULE$.apply( - Arrays.asList(4, 1, 5, 6, 2, 7, 3, 8, 9, 10), Arrays.asList(1, 2, 3)), + Arrays.asList(4, 1, 5, 6, 2, 7, 3, 8, 9, 10), Arrays.asList(1, 2, 3)), Tuple2$.MODULE$.apply( - Arrays.asList(1, 2, 3, 4, 5), Arrays.asList())), 2); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; + Arrays.asList(1, 2, 3, 4, 5), Arrays.asList())), 2); } @Test diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java index 8a320afa4b13..05128ea34342 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java @@ -17,39 +17,24 @@ package org.apache.spark.mllib.feature; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; -public class JavaTfIdfSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaTfIdfSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaTfIdfSuite extends SharedSparkSession { @Test public void tfIdf() { // The tests are to check Java compatibility. HashingTF tf = new HashingTF(); @SuppressWarnings("unchecked") - JavaRDD> documents = sc.parallelize(Arrays.asList( + JavaRDD> documents = jsc.parallelize(Arrays.asList( Arrays.asList("this is a sentence".split(" ")), Arrays.asList("this is another sentence".split(" ")), Arrays.asList("this is still a sentence".split(" "))), 2); @@ -59,7 +44,7 @@ public void tfIdf() { JavaRDD tfIdfs = idf.fit(termFreqs).transform(termFreqs); List localTfIdfs = tfIdfs.collect(); int indexOfThis = tf.indexOf("this"); - for (Vector v: localTfIdfs) { + for (Vector v : localTfIdfs) { Assert.assertEquals(0.0, v.apply(indexOfThis), 1e-15); } } @@ -69,7 +54,7 @@ public void tfIdfMinimumDocumentFrequency() { // The tests are to check Java compatibility. HashingTF tf = new HashingTF(); @SuppressWarnings("unchecked") - JavaRDD> documents = sc.parallelize(Arrays.asList( + JavaRDD> documents = jsc.parallelize(Arrays.asList( Arrays.asList("this is a sentence".split(" ")), Arrays.asList("this is another sentence".split(" ")), Arrays.asList("this is still a sentence".split(" "))), 2); @@ -79,7 +64,7 @@ public void tfIdfMinimumDocumentFrequency() { JavaRDD tfIdfs = idf.fit(termFreqs).transform(termFreqs); List localTfIdfs = tfIdfs.collect(); int indexOfThis = tf.indexOf("this"); - for (Vector v: localTfIdfs) { + for (Vector v : localTfIdfs) { Assert.assertEquals(0.0, v.apply(indexOfThis), 1e-15); } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java index e13ed07e283d..3e3abddbee63 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java @@ -17,34 +17,20 @@ package org.apache.spark.mllib.feature; -import java.io.Serializable; import java.util.Arrays; import java.util.List; +import com.google.common.base.Strings; + import scala.Tuple2; -import com.google.common.base.Strings; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; - -public class JavaWord2VecSuite implements Serializable { - private transient JavaSparkContext sc; - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaWord2VecSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaWord2VecSuite extends SharedSparkSession { @Test @SuppressWarnings("unchecked") @@ -53,7 +39,7 @@ public void word2Vec() { String sentence = Strings.repeat("a b ", 100) + Strings.repeat("a c ", 10); List words = Arrays.asList(sentence.split(" ")); List> localDoc = Arrays.asList(words, words); - JavaRDD> doc = sc.parallelize(localDoc); + JavaRDD> doc = jsc.parallelize(localDoc); Word2Vec word2vec = new Word2Vec() .setVectorSize(10) .setSeed(42L); diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java index 2bef7a860975..3451e0773759 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java @@ -16,42 +16,26 @@ */ package org.apache.spark.mllib.fpm; -import java.io.Serializable; import java.util.Arrays; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset; -public class JavaAssociationRulesSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaFPGrowth"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaAssociationRulesSuite extends SharedSparkSession { @Test public void runAssociationRules() { @SuppressWarnings("unchecked") - JavaRDD> freqItemsets = sc.parallelize(Arrays.asList( - new FreqItemset(new String[] {"a"}, 15L), - new FreqItemset(new String[] {"b"}, 35L), - new FreqItemset(new String[] {"a", "b"}, 12L) + JavaRDD> freqItemsets = jsc.parallelize(Arrays.asList( + new FreqItemset(new String[]{"a"}, 15L), + new FreqItemset(new String[]{"b"}, 35L), + new FreqItemset(new String[]{"a", "b"}, 12L) )); JavaRDD> results = (new AssociationRules()).run(freqItemsets); } } - diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java index 916fff14a721..46e9dd8b5982 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java @@ -18,38 +18,24 @@ package org.apache.spark.mllib.fpm; import java.io.File; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; +import static org.junit.Assert.assertEquals; + import org.junit.Test; -import static org.junit.Assert.*; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.util.Utils; -public class JavaFPGrowthSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaFPGrowth"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaFPGrowthSuite extends SharedSparkSession { @Test public void runFPGrowth() { @SuppressWarnings("unchecked") - JavaRDD> rdd = sc.parallelize(Arrays.asList( + JavaRDD> rdd = jsc.parallelize(Arrays.asList( Arrays.asList("r z h k p".split(" ")), Arrays.asList("z y x w v u t s".split(" ")), Arrays.asList("s x o n r".split(" ")), @@ -65,7 +51,7 @@ public void runFPGrowth() { List> freqItemsets = model.freqItemsets().toJavaRDD().collect(); assertEquals(18, freqItemsets.size()); - for (FPGrowth.FreqItemset itemset: freqItemsets) { + for (FPGrowth.FreqItemset itemset : freqItemsets) { // Test return types. List items = itemset.javaItems(); long freq = itemset.freq(); @@ -76,7 +62,7 @@ public void runFPGrowth() { public void runFPGrowthSaveLoad() { @SuppressWarnings("unchecked") - JavaRDD> rdd = sc.parallelize(Arrays.asList( + JavaRDD> rdd = jsc.parallelize(Arrays.asList( Arrays.asList("r z h k p".split(" ")), Arrays.asList("z y x w v u t s".split(" ")), Arrays.asList("s x o n r".split(" ")), @@ -94,15 +80,15 @@ public void runFPGrowthSaveLoad() { String outputPath = tempDir.getPath(); try { - model.save(sc.sc(), outputPath); + model.save(spark.sparkContext(), outputPath); @SuppressWarnings("unchecked") FPGrowthModel newModel = - (FPGrowthModel) FPGrowthModel.load(sc.sc(), outputPath); + (FPGrowthModel) FPGrowthModel.load(spark.sparkContext(), outputPath); List> freqItemsets = newModel.freqItemsets().toJavaRDD() .collect(); assertEquals(18, freqItemsets.size()); - for (FPGrowth.FreqItemset itemset: freqItemsets) { + for (FPGrowth.FreqItemset itemset : freqItemsets) { // Test return types. List items = itemset.javaItems(); long freq = itemset.freq(); diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java index 8a67793abc14..75b0ec648019 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java @@ -21,33 +21,19 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.fpm.PrefixSpan.FreqSequence; import org.apache.spark.util.Utils; -public class JavaPrefixSpanSuite { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaPrefixSpan"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaPrefixSpanSuite extends SharedSparkSession { @Test public void runPrefixSpan() { - JavaRDD>> sequences = sc.parallelize(Arrays.asList( + JavaRDD>> sequences = jsc.parallelize(Arrays.asList( Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3)), Arrays.asList(Arrays.asList(1), Arrays.asList(3, 2), Arrays.asList(1, 2)), Arrays.asList(Arrays.asList(1, 2), Arrays.asList(5)), @@ -61,7 +47,7 @@ public void runPrefixSpan() { List> localFreqSeqs = freqSeqs.collect(); Assert.assertEquals(5, localFreqSeqs.size()); // Check that each frequent sequence could be materialized. - for (PrefixSpan.FreqSequence freqSeq: localFreqSeqs) { + for (PrefixSpan.FreqSequence freqSeq : localFreqSeqs) { List> seq = freqSeq.javaSequence(); long freq = freqSeq.freq(); } @@ -69,7 +55,7 @@ public void runPrefixSpan() { @Test public void runPrefixSpanSaveLoad() { - JavaRDD>> sequences = sc.parallelize(Arrays.asList( + JavaRDD>> sequences = jsc.parallelize(Arrays.asList( Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3)), Arrays.asList(Arrays.asList(1), Arrays.asList(3, 2), Arrays.asList(1, 2)), Arrays.asList(Arrays.asList(1, 2), Arrays.asList(5)), @@ -85,13 +71,13 @@ public void runPrefixSpanSaveLoad() { String outputPath = tempDir.getPath(); try { - model.save(sc.sc(), outputPath); - PrefixSpanModel newModel = PrefixSpanModel.load(sc.sc(), outputPath); + model.save(spark.sparkContext(), outputPath); + PrefixSpanModel newModel = PrefixSpanModel.load(spark.sparkContext(), outputPath); JavaRDD> freqSeqs = newModel.freqSequences().toJavaRDD(); List> localFreqSeqs = freqSeqs.collect(); Assert.assertEquals(5, localFreqSeqs.size()); // Check that each frequent sequence could be materialized. - for (PrefixSpan.FreqSequence freqSeq: localFreqSeqs) { + for (PrefixSpan.FreqSequence freqSeq : localFreqSeqs) { List> seq = freqSeq.javaSequence(); long freq = freqSeq.freq(); } diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java index 8beea102efd0..f427846b9ad1 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java @@ -17,147 +17,148 @@ package org.apache.spark.mllib.linalg; -import static org.junit.Assert.*; -import org.junit.Test; - -import java.io.Serializable; import java.util.Random; -public class JavaMatricesSuite implements Serializable { - - @Test - public void randMatrixConstruction() { - Random rng = new Random(24); - Matrix r = Matrices.rand(3, 4, rng); - rng.setSeed(24); - DenseMatrix dr = DenseMatrix.rand(3, 4, rng); - assertArrayEquals(r.toArray(), dr.toArray(), 0.0); - - rng.setSeed(24); - Matrix rn = Matrices.randn(3, 4, rng); - rng.setSeed(24); - DenseMatrix drn = DenseMatrix.randn(3, 4, rng); - assertArrayEquals(rn.toArray(), drn.toArray(), 0.0); - - rng.setSeed(24); - Matrix s = Matrices.sprand(3, 4, 0.5, rng); - rng.setSeed(24); - SparseMatrix sr = SparseMatrix.sprand(3, 4, 0.5, rng); - assertArrayEquals(s.toArray(), sr.toArray(), 0.0); - - rng.setSeed(24); - Matrix sn = Matrices.sprandn(3, 4, 0.5, rng); - rng.setSeed(24); - SparseMatrix srn = SparseMatrix.sprandn(3, 4, 0.5, rng); - assertArrayEquals(sn.toArray(), srn.toArray(), 0.0); - } - - @Test - public void identityMatrixConstruction() { - Matrix r = Matrices.eye(2); - DenseMatrix dr = DenseMatrix.eye(2); - SparseMatrix sr = SparseMatrix.speye(2); - assertArrayEquals(r.toArray(), dr.toArray(), 0.0); - assertArrayEquals(sr.toArray(), dr.toArray(), 0.0); - assertArrayEquals(r.toArray(), new double[]{1.0, 0.0, 0.0, 1.0}, 0.0); - } - - @Test - public void diagonalMatrixConstruction() { - Vector v = Vectors.dense(1.0, 0.0, 2.0); - Vector sv = Vectors.sparse(3, new int[]{0, 2}, new double[]{1.0, 2.0}); - - Matrix m = Matrices.diag(v); - Matrix sm = Matrices.diag(sv); - DenseMatrix d = DenseMatrix.diag(v); - DenseMatrix sd = DenseMatrix.diag(sv); - SparseMatrix s = SparseMatrix.spdiag(v); - SparseMatrix ss = SparseMatrix.spdiag(sv); - - assertArrayEquals(m.toArray(), sm.toArray(), 0.0); - assertArrayEquals(d.toArray(), sm.toArray(), 0.0); - assertArrayEquals(d.toArray(), sd.toArray(), 0.0); - assertArrayEquals(sd.toArray(), s.toArray(), 0.0); - assertArrayEquals(s.toArray(), ss.toArray(), 0.0); - assertArrayEquals(s.values(), ss.values(), 0.0); - assertEquals(2, s.values().length); - assertEquals(2, ss.values().length); - assertEquals(4, s.colPtrs().length); - assertEquals(4, ss.colPtrs().length); - } - - @Test - public void zerosMatrixConstruction() { - Matrix z = Matrices.zeros(2, 2); - Matrix one = Matrices.ones(2, 2); - DenseMatrix dz = DenseMatrix.zeros(2, 2); - DenseMatrix done = DenseMatrix.ones(2, 2); - - assertArrayEquals(z.toArray(), new double[]{0.0, 0.0, 0.0, 0.0}, 0.0); - assertArrayEquals(dz.toArray(), new double[]{0.0, 0.0, 0.0, 0.0}, 0.0); - assertArrayEquals(one.toArray(), new double[]{1.0, 1.0, 1.0, 1.0}, 0.0); - assertArrayEquals(done.toArray(), new double[]{1.0, 1.0, 1.0, 1.0}, 0.0); - } - - @Test - public void sparseDenseConversion() { - int m = 3; - int n = 2; - double[] values = new double[]{1.0, 2.0, 4.0, 5.0}; - double[] allValues = new double[]{1.0, 2.0, 0.0, 0.0, 4.0, 5.0}; - int[] colPtrs = new int[]{0, 2, 4}; - int[] rowIndices = new int[]{0, 1, 1, 2}; - - SparseMatrix spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values); - DenseMatrix deMat1 = new DenseMatrix(m, n, allValues); - - SparseMatrix spMat2 = deMat1.toSparse(); - DenseMatrix deMat2 = spMat1.toDense(); - - assertArrayEquals(spMat1.toArray(), spMat2.toArray(), 0.0); - assertArrayEquals(deMat1.toArray(), deMat2.toArray(), 0.0); - } - - @Test - public void concatenateMatrices() { - int m = 3; - int n = 2; - - Random rng = new Random(42); - SparseMatrix spMat1 = SparseMatrix.sprand(m, n, 0.5, rng); - rng.setSeed(42); - DenseMatrix deMat1 = DenseMatrix.rand(m, n, rng); - Matrix deMat2 = Matrices.eye(3); - Matrix spMat2 = Matrices.speye(3); - Matrix deMat3 = Matrices.eye(2); - Matrix spMat3 = Matrices.speye(2); - - Matrix spHorz = Matrices.horzcat(new Matrix[]{spMat1, spMat2}); - Matrix deHorz1 = Matrices.horzcat(new Matrix[]{deMat1, deMat2}); - Matrix deHorz2 = Matrices.horzcat(new Matrix[]{spMat1, deMat2}); - Matrix deHorz3 = Matrices.horzcat(new Matrix[]{deMat1, spMat2}); - - assertEquals(3, deHorz1.numRows()); - assertEquals(3, deHorz2.numRows()); - assertEquals(3, deHorz3.numRows()); - assertEquals(3, spHorz.numRows()); - assertEquals(5, deHorz1.numCols()); - assertEquals(5, deHorz2.numCols()); - assertEquals(5, deHorz3.numCols()); - assertEquals(5, spHorz.numCols()); - - Matrix spVert = Matrices.vertcat(new Matrix[]{spMat1, spMat3}); - Matrix deVert1 = Matrices.vertcat(new Matrix[]{deMat1, deMat3}); - Matrix deVert2 = Matrices.vertcat(new Matrix[]{spMat1, deMat3}); - Matrix deVert3 = Matrices.vertcat(new Matrix[]{deMat1, spMat3}); - - assertEquals(5, deVert1.numRows()); - assertEquals(5, deVert2.numRows()); - assertEquals(5, deVert3.numRows()); - assertEquals(5, spVert.numRows()); - assertEquals(2, deVert1.numCols()); - assertEquals(2, deVert2.numCols()); - assertEquals(2, deVert3.numCols()); - assertEquals(2, spVert.numCols()); - } +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +public class JavaMatricesSuite { + + @Test + public void randMatrixConstruction() { + Random rng = new Random(24); + Matrix r = Matrices.rand(3, 4, rng); + rng.setSeed(24); + DenseMatrix dr = DenseMatrix.rand(3, 4, rng); + assertArrayEquals(r.toArray(), dr.toArray(), 0.0); + + rng.setSeed(24); + Matrix rn = Matrices.randn(3, 4, rng); + rng.setSeed(24); + DenseMatrix drn = DenseMatrix.randn(3, 4, rng); + assertArrayEquals(rn.toArray(), drn.toArray(), 0.0); + + rng.setSeed(24); + Matrix s = Matrices.sprand(3, 4, 0.5, rng); + rng.setSeed(24); + SparseMatrix sr = SparseMatrix.sprand(3, 4, 0.5, rng); + assertArrayEquals(s.toArray(), sr.toArray(), 0.0); + + rng.setSeed(24); + Matrix sn = Matrices.sprandn(3, 4, 0.5, rng); + rng.setSeed(24); + SparseMatrix srn = SparseMatrix.sprandn(3, 4, 0.5, rng); + assertArrayEquals(sn.toArray(), srn.toArray(), 0.0); + } + + @Test + public void identityMatrixConstruction() { + Matrix r = Matrices.eye(2); + DenseMatrix dr = DenseMatrix.eye(2); + SparseMatrix sr = SparseMatrix.speye(2); + assertArrayEquals(r.toArray(), dr.toArray(), 0.0); + assertArrayEquals(sr.toArray(), dr.toArray(), 0.0); + assertArrayEquals(r.toArray(), new double[]{1.0, 0.0, 0.0, 1.0}, 0.0); + } + + @Test + public void diagonalMatrixConstruction() { + Vector v = Vectors.dense(1.0, 0.0, 2.0); + Vector sv = Vectors.sparse(3, new int[]{0, 2}, new double[]{1.0, 2.0}); + + Matrix m = Matrices.diag(v); + Matrix sm = Matrices.diag(sv); + DenseMatrix d = DenseMatrix.diag(v); + DenseMatrix sd = DenseMatrix.diag(sv); + SparseMatrix s = SparseMatrix.spdiag(v); + SparseMatrix ss = SparseMatrix.spdiag(sv); + + assertArrayEquals(m.toArray(), sm.toArray(), 0.0); + assertArrayEquals(d.toArray(), sm.toArray(), 0.0); + assertArrayEquals(d.toArray(), sd.toArray(), 0.0); + assertArrayEquals(sd.toArray(), s.toArray(), 0.0); + assertArrayEquals(s.toArray(), ss.toArray(), 0.0); + assertArrayEquals(s.values(), ss.values(), 0.0); + assertEquals(2, s.values().length); + assertEquals(2, ss.values().length); + assertEquals(4, s.colPtrs().length); + assertEquals(4, ss.colPtrs().length); + } + + @Test + public void zerosMatrixConstruction() { + Matrix z = Matrices.zeros(2, 2); + Matrix one = Matrices.ones(2, 2); + DenseMatrix dz = DenseMatrix.zeros(2, 2); + DenseMatrix done = DenseMatrix.ones(2, 2); + + assertArrayEquals(z.toArray(), new double[]{0.0, 0.0, 0.0, 0.0}, 0.0); + assertArrayEquals(dz.toArray(), new double[]{0.0, 0.0, 0.0, 0.0}, 0.0); + assertArrayEquals(one.toArray(), new double[]{1.0, 1.0, 1.0, 1.0}, 0.0); + assertArrayEquals(done.toArray(), new double[]{1.0, 1.0, 1.0, 1.0}, 0.0); + } + + @Test + public void sparseDenseConversion() { + int m = 3; + int n = 2; + double[] values = new double[]{1.0, 2.0, 4.0, 5.0}; + double[] allValues = new double[]{1.0, 2.0, 0.0, 0.0, 4.0, 5.0}; + int[] colPtrs = new int[]{0, 2, 4}; + int[] rowIndices = new int[]{0, 1, 1, 2}; + + SparseMatrix spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values); + DenseMatrix deMat1 = new DenseMatrix(m, n, allValues); + + SparseMatrix spMat2 = deMat1.toSparse(); + DenseMatrix deMat2 = spMat1.toDense(); + + assertArrayEquals(spMat1.toArray(), spMat2.toArray(), 0.0); + assertArrayEquals(deMat1.toArray(), deMat2.toArray(), 0.0); + } + + @Test + public void concatenateMatrices() { + int m = 3; + int n = 2; + + Random rng = new Random(42); + SparseMatrix spMat1 = SparseMatrix.sprand(m, n, 0.5, rng); + rng.setSeed(42); + DenseMatrix deMat1 = DenseMatrix.rand(m, n, rng); + Matrix deMat2 = Matrices.eye(3); + Matrix spMat2 = Matrices.speye(3); + Matrix deMat3 = Matrices.eye(2); + Matrix spMat3 = Matrices.speye(2); + + Matrix spHorz = Matrices.horzcat(new Matrix[]{spMat1, spMat2}); + Matrix deHorz1 = Matrices.horzcat(new Matrix[]{deMat1, deMat2}); + Matrix deHorz2 = Matrices.horzcat(new Matrix[]{spMat1, deMat2}); + Matrix deHorz3 = Matrices.horzcat(new Matrix[]{deMat1, spMat2}); + + assertEquals(3, deHorz1.numRows()); + assertEquals(3, deHorz2.numRows()); + assertEquals(3, deHorz3.numRows()); + assertEquals(3, spHorz.numRows()); + assertEquals(5, deHorz1.numCols()); + assertEquals(5, deHorz2.numCols()); + assertEquals(5, deHorz3.numCols()); + assertEquals(5, spHorz.numCols()); + + Matrix spVert = Matrices.vertcat(new Matrix[]{spMat1, spMat3}); + Matrix deVert1 = Matrices.vertcat(new Matrix[]{deMat1, deMat3}); + Matrix deVert2 = Matrices.vertcat(new Matrix[]{spMat1, deMat3}); + Matrix deVert3 = Matrices.vertcat(new Matrix[]{deMat1, spMat3}); + + assertEquals(5, deVert1.numRows()); + assertEquals(5, deVert2.numRows()); + assertEquals(5, deVert3.numRows()); + assertEquals(5, spVert.numRows()); + assertEquals(2, deVert1.numCols()); + assertEquals(2, deVert2.numCols()); + assertEquals(2, deVert3.numCols()); + assertEquals(2, spVert.numCols()); + } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java index 4ba8e543a9a6..f67f555e418a 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java @@ -17,15 +17,15 @@ package org.apache.spark.mllib.linalg; -import java.io.Serializable; import java.util.Arrays; +import static org.junit.Assert.assertArrayEquals; + import scala.Tuple2; import org.junit.Test; -import static org.junit.Assert.*; -public class JavaVectorsSuite implements Serializable { +public class JavaVectorsSuite { @Test public void denseArrayConstruction() { @@ -37,8 +37,8 @@ public void denseArrayConstruction() { public void sparseArrayConstruction() { @SuppressWarnings("unchecked") Vector v = Vectors.sparse(3, Arrays.asList( - new Tuple2<>(0, 2.0), - new Tuple2<>(2, 3.0))); + new Tuple2<>(0, 2.0), + new Tuple2<>(2, 3.0))); assertArrayEquals(new double[]{2.0, 0.0, 3.0}, v.toArray(), 0.0); } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java index be58691f4d87..6d114024c31b 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java @@ -20,40 +20,26 @@ import java.io.Serializable; import java.util.Arrays; -import org.apache.spark.api.java.JavaRDD; import org.junit.Assert; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaDoubleRDD; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.mllib.linalg.Vector; import static org.apache.spark.mllib.random.RandomRDDs.*; -public class JavaRandomRDDsSuite { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaRandomRDDsSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaRandomRDDsSuite extends SharedSparkSession { @Test public void testUniformRDD() { long m = 1000L; int p = 2; long seed = 1L; - JavaDoubleRDD rdd1 = uniformJavaRDD(sc, m); - JavaDoubleRDD rdd2 = uniformJavaRDD(sc, m, p); - JavaDoubleRDD rdd3 = uniformJavaRDD(sc, m, p, seed); - for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaDoubleRDD rdd1 = uniformJavaRDD(jsc, m); + JavaDoubleRDD rdd2 = uniformJavaRDD(jsc, m, p); + JavaDoubleRDD rdd3 = uniformJavaRDD(jsc, m, p, seed); + for (JavaDoubleRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -63,10 +49,10 @@ public void testNormalRDD() { long m = 1000L; int p = 2; long seed = 1L; - JavaDoubleRDD rdd1 = normalJavaRDD(sc, m); - JavaDoubleRDD rdd2 = normalJavaRDD(sc, m, p); - JavaDoubleRDD rdd3 = normalJavaRDD(sc, m, p, seed); - for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaDoubleRDD rdd1 = normalJavaRDD(jsc, m); + JavaDoubleRDD rdd2 = normalJavaRDD(jsc, m, p); + JavaDoubleRDD rdd3 = normalJavaRDD(jsc, m, p, seed); + for (JavaDoubleRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -78,10 +64,10 @@ public void testLNormalRDD() { long m = 1000L; int p = 2; long seed = 1L; - JavaDoubleRDD rdd1 = logNormalJavaRDD(sc, mean, std, m); - JavaDoubleRDD rdd2 = logNormalJavaRDD(sc, mean, std, m, p); - JavaDoubleRDD rdd3 = logNormalJavaRDD(sc, mean, std, m, p, seed); - for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaDoubleRDD rdd1 = logNormalJavaRDD(jsc, mean, std, m); + JavaDoubleRDD rdd2 = logNormalJavaRDD(jsc, mean, std, m, p); + JavaDoubleRDD rdd3 = logNormalJavaRDD(jsc, mean, std, m, p, seed); + for (JavaDoubleRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -92,10 +78,10 @@ public void testPoissonRDD() { long m = 1000L; int p = 2; long seed = 1L; - JavaDoubleRDD rdd1 = poissonJavaRDD(sc, mean, m); - JavaDoubleRDD rdd2 = poissonJavaRDD(sc, mean, m, p); - JavaDoubleRDD rdd3 = poissonJavaRDD(sc, mean, m, p, seed); - for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaDoubleRDD rdd1 = poissonJavaRDD(jsc, mean, m); + JavaDoubleRDD rdd2 = poissonJavaRDD(jsc, mean, m, p); + JavaDoubleRDD rdd3 = poissonJavaRDD(jsc, mean, m, p, seed); + for (JavaDoubleRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -106,10 +92,10 @@ public void testExponentialRDD() { long m = 1000L; int p = 2; long seed = 1L; - JavaDoubleRDD rdd1 = exponentialJavaRDD(sc, mean, m); - JavaDoubleRDD rdd2 = exponentialJavaRDD(sc, mean, m, p); - JavaDoubleRDD rdd3 = exponentialJavaRDD(sc, mean, m, p, seed); - for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaDoubleRDD rdd1 = exponentialJavaRDD(jsc, mean, m); + JavaDoubleRDD rdd2 = exponentialJavaRDD(jsc, mean, m, p); + JavaDoubleRDD rdd3 = exponentialJavaRDD(jsc, mean, m, p, seed); + for (JavaDoubleRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -117,14 +103,14 @@ public void testExponentialRDD() { @Test public void testGammaRDD() { double shape = 1.0; - double scale = 2.0; + double jscale = 2.0; long m = 1000L; int p = 2; long seed = 1L; - JavaDoubleRDD rdd1 = gammaJavaRDD(sc, shape, scale, m); - JavaDoubleRDD rdd2 = gammaJavaRDD(sc, shape, scale, m, p); - JavaDoubleRDD rdd3 = gammaJavaRDD(sc, shape, scale, m, p, seed); - for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaDoubleRDD rdd1 = gammaJavaRDD(jsc, shape, jscale, m); + JavaDoubleRDD rdd2 = gammaJavaRDD(jsc, shape, jscale, m, p); + JavaDoubleRDD rdd3 = gammaJavaRDD(jsc, shape, jscale, m, p, seed); + for (JavaDoubleRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -137,10 +123,10 @@ public void testUniformVectorRDD() { int n = 10; int p = 2; long seed = 1L; - JavaRDD rdd1 = uniformJavaVectorRDD(sc, m, n); - JavaRDD rdd2 = uniformJavaVectorRDD(sc, m, n, p); - JavaRDD rdd3 = uniformJavaVectorRDD(sc, m, n, p, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = uniformJavaVectorRDD(jsc, m, n); + JavaRDD rdd2 = uniformJavaVectorRDD(jsc, m, n, p); + JavaRDD rdd3 = uniformJavaVectorRDD(jsc, m, n, p, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -153,10 +139,10 @@ public void testNormalVectorRDD() { int n = 10; int p = 2; long seed = 1L; - JavaRDD rdd1 = normalJavaVectorRDD(sc, m, n); - JavaRDD rdd2 = normalJavaVectorRDD(sc, m, n, p); - JavaRDD rdd3 = normalJavaVectorRDD(sc, m, n, p, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = normalJavaVectorRDD(jsc, m, n); + JavaRDD rdd2 = normalJavaVectorRDD(jsc, m, n, p); + JavaRDD rdd3 = normalJavaVectorRDD(jsc, m, n, p, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -171,10 +157,10 @@ public void testLogNormalVectorRDD() { int n = 10; int p = 2; long seed = 1L; - JavaRDD rdd1 = logNormalJavaVectorRDD(sc, mean, std, m, n); - JavaRDD rdd2 = logNormalJavaVectorRDD(sc, mean, std, m, n, p); - JavaRDD rdd3 = logNormalJavaVectorRDD(sc, mean, std, m, n, p, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = logNormalJavaVectorRDD(jsc, mean, std, m, n); + JavaRDD rdd2 = logNormalJavaVectorRDD(jsc, mean, std, m, n, p); + JavaRDD rdd3 = logNormalJavaVectorRDD(jsc, mean, std, m, n, p, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -188,10 +174,10 @@ public void testPoissonVectorRDD() { int n = 10; int p = 2; long seed = 1L; - JavaRDD rdd1 = poissonJavaVectorRDD(sc, mean, m, n); - JavaRDD rdd2 = poissonJavaVectorRDD(sc, mean, m, n, p); - JavaRDD rdd3 = poissonJavaVectorRDD(sc, mean, m, n, p, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = poissonJavaVectorRDD(jsc, mean, m, n); + JavaRDD rdd2 = poissonJavaVectorRDD(jsc, mean, m, n, p); + JavaRDD rdd3 = poissonJavaVectorRDD(jsc, mean, m, n, p, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -205,10 +191,10 @@ public void testExponentialVectorRDD() { int n = 10; int p = 2; long seed = 1L; - JavaRDD rdd1 = exponentialJavaVectorRDD(sc, mean, m, n); - JavaRDD rdd2 = exponentialJavaVectorRDD(sc, mean, m, n, p); - JavaRDD rdd3 = exponentialJavaVectorRDD(sc, mean, m, n, p, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = exponentialJavaVectorRDD(jsc, mean, m, n); + JavaRDD rdd2 = exponentialJavaVectorRDD(jsc, mean, m, n, p); + JavaRDD rdd3 = exponentialJavaVectorRDD(jsc, mean, m, n, p, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -218,15 +204,15 @@ public void testExponentialVectorRDD() { @SuppressWarnings("unchecked") public void testGammaVectorRDD() { double shape = 1.0; - double scale = 2.0; + double jscale = 2.0; long m = 100L; int n = 10; int p = 2; long seed = 1L; - JavaRDD rdd1 = gammaJavaVectorRDD(sc, shape, scale, m, n); - JavaRDD rdd2 = gammaJavaVectorRDD(sc, shape, scale, m, n, p); - JavaRDD rdd3 = gammaJavaVectorRDD(sc, shape, scale, m, n, p, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = gammaJavaVectorRDD(jsc, shape, jscale, m, n); + JavaRDD rdd2 = gammaJavaVectorRDD(jsc, shape, jscale, m, n, p); + JavaRDD rdd3 = gammaJavaVectorRDD(jsc, shape, jscale, m, n, p, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -238,10 +224,10 @@ public void testArbitrary() { long seed = 1L; int numPartitions = 0; StringGenerator gen = new StringGenerator(); - JavaRDD rdd1 = randomJavaRDD(sc, gen, size); - JavaRDD rdd2 = randomJavaRDD(sc, gen, size, numPartitions); - JavaRDD rdd3 = randomJavaRDD(sc, gen, size, numPartitions, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = randomJavaRDD(jsc, gen, size); + JavaRDD rdd2 = randomJavaRDD(jsc, gen, size, numPartitions); + JavaRDD rdd3 = randomJavaRDD(jsc, gen, size, numPartitions, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(size, rdd.count()); Assert.assertEquals(2, rdd.first().length()); } @@ -255,10 +241,10 @@ public void testRandomVectorRDD() { int n = 10; int p = 2; long seed = 1L; - JavaRDD rdd1 = randomJavaVectorRDD(sc, generator, m, n); - JavaRDD rdd2 = randomJavaVectorRDD(sc, generator, m, n, p); - JavaRDD rdd3 = randomJavaVectorRDD(sc, generator, m, n, p, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = randomJavaVectorRDD(jsc, generator, m, n); + JavaRDD rdd2 = randomJavaVectorRDD(jsc, generator, m, n, p); + JavaRDD rdd3 = randomJavaVectorRDD(jsc, generator, m, n, p, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -271,10 +257,12 @@ class StringGenerator implements RandomDataGenerator, Serializable { public String nextValue() { return "42"; } + @Override public StringGenerator copy() { return new StringGenerator(); } + @Override public void setSeed(long seed) { } diff --git a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java index d0bf7f556dcc..363ab42546d1 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java @@ -17,55 +17,40 @@ package org.apache.spark.mllib.recommendation; -import java.io.Serializable; import java.util.ArrayList; import java.util.List; import scala.Tuple2; import scala.Tuple3; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class JavaALSSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaALS"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaALSSuite extends SharedSparkSession { private void validatePrediction( - MatrixFactorizationModel model, - int users, - int products, - double[] trueRatings, - double matchThreshold, - boolean implicitPrefs, - double[] truePrefs) { + MatrixFactorizationModel model, + int users, + int products, + double[] trueRatings, + double matchThreshold, + boolean implicitPrefs, + double[] truePrefs) { List> localUsersProducts = new ArrayList<>(users * products); - for (int u=0; u < users; ++u) { - for (int p=0; p < products; ++p) { + for (int u = 0; u < users; ++u) { + for (int p = 0; p < products; ++p) { localUsersProducts.add(new Tuple2<>(u, p)); } } - JavaPairRDD usersProducts = sc.parallelizePairs(localUsersProducts); + JavaPairRDD usersProducts = jsc.parallelizePairs(localUsersProducts); List predictedRatings = model.predict(usersProducts).collect(); Assert.assertEquals(users * products, predictedRatings.size()); if (!implicitPrefs) { - for (Rating r: predictedRatings) { + for (Rating r : predictedRatings) { double prediction = r.rating(); double correct = trueRatings[r.product() * users + r.user()]; Assert.assertTrue(String.format("Prediction=%2.4f not below match threshold of %2.2f", @@ -76,7 +61,7 @@ private void validatePrediction( // (ref Mahout's implicit ALS tests) double sqErr = 0.0; double denom = 0.0; - for (Rating r: predictedRatings) { + for (Rating r : predictedRatings) { double prediction = r.rating(); double truePref = truePrefs[r.product() * users + r.user()]; double confidence = 1.0 + @@ -98,9 +83,9 @@ public void runALSUsingStaticMethods() { int users = 50; int products = 100; Tuple3, double[], double[]> testData = - ALSSuite.generateRatingsAsJava(users, products, features, 0.7, false, false); + ALSSuite.generateRatingsAsJava(users, products, features, 0.7, false, false); - JavaRDD data = sc.parallelize(testData._1()); + JavaRDD data = jsc.parallelize(testData._1()); MatrixFactorizationModel model = ALS.train(data.rdd(), features, iterations); validatePrediction(model, users, products, testData._2(), 0.3, false, testData._3()); } @@ -112,9 +97,9 @@ public void runALSUsingConstructor() { int users = 100; int products = 200; Tuple3, double[], double[]> testData = - ALSSuite.generateRatingsAsJava(users, products, features, 0.7, false, false); + ALSSuite.generateRatingsAsJava(users, products, features, 0.7, false, false); - JavaRDD data = sc.parallelize(testData._1()); + JavaRDD data = jsc.parallelize(testData._1()); MatrixFactorizationModel model = new ALS().setRank(features) .setIterations(iterations) @@ -129,9 +114,9 @@ public void runImplicitALSUsingStaticMethods() { int users = 80; int products = 160; Tuple3, double[], double[]> testData = - ALSSuite.generateRatingsAsJava(users, products, features, 0.7, true, false); + ALSSuite.generateRatingsAsJava(users, products, features, 0.7, true, false); - JavaRDD data = sc.parallelize(testData._1()); + JavaRDD data = jsc.parallelize(testData._1()); MatrixFactorizationModel model = ALS.trainImplicit(data.rdd(), features, iterations); validatePrediction(model, users, products, testData._2(), 0.4, true, testData._3()); } @@ -143,9 +128,9 @@ public void runImplicitALSUsingConstructor() { int users = 100; int products = 200; Tuple3, double[], double[]> testData = - ALSSuite.generateRatingsAsJava(users, products, features, 0.7, true, false); + ALSSuite.generateRatingsAsJava(users, products, features, 0.7, true, false); - JavaRDD data = sc.parallelize(testData._1()); + JavaRDD data = jsc.parallelize(testData._1()); MatrixFactorizationModel model = new ALS().setRank(features) .setIterations(iterations) @@ -161,9 +146,9 @@ public void runImplicitALSWithNegativeWeight() { int users = 80; int products = 160; Tuple3, double[], double[]> testData = - ALSSuite.generateRatingsAsJava(users, products, features, 0.7, true, true); + ALSSuite.generateRatingsAsJava(users, products, features, 0.7, true, true); - JavaRDD data = sc.parallelize(testData._1()); + JavaRDD data = jsc.parallelize(testData._1()); MatrixFactorizationModel model = new ALS().setRank(features) .setIterations(iterations) .setImplicitPrefs(true) @@ -179,8 +164,8 @@ public void runRecommend() { int users = 200; int products = 50; List testData = ALSSuite.generateRatingsAsJava( - users, products, features, 0.7, true, false)._1(); - JavaRDD data = sc.parallelize(testData); + users, products, features, 0.7, true, false)._1(); + JavaRDD data = jsc.parallelize(testData); MatrixFactorizationModel model = new ALS().setRank(features) .setIterations(iterations) .setImplicitPrefs(true) @@ -193,7 +178,7 @@ public void runRecommend() { private static void validateRecommendations(Rating[] recommendations, int howMany) { Assert.assertEquals(howMany, recommendations.length); for (int i = 1; i < recommendations.length; i++) { - Assert.assertTrue(recommendations[i-1].rating() >= recommendations[i].rating()); + Assert.assertTrue(recommendations[i - 1].rating() >= recommendations[i].rating()); } Assert.assertTrue(recommendations[0].rating() > 0.7); } diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java index 3db9b39e740e..dbd4cbfd2b74 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java @@ -17,30 +17,26 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import scala.Tuple3; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaDoubleRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class JavaIsotonicRegressionSuite implements Serializable { - private transient JavaSparkContext sc; +public class JavaIsotonicRegressionSuite extends SharedSparkSession { private static List> generateIsotonicInput(double[] labels) { List> input = new ArrayList<>(labels.length); for (int i = 1; i <= labels.length; i++) { - input.add(new Tuple3<>(labels[i-1], (double) i, 1.0)); + input.add(new Tuple3<>(labels[i - 1], (double) i, 1.0)); } return input; @@ -48,29 +44,18 @@ private static List> generateIsotonicInput(double private IsotonicRegressionModel runIsotonicRegression(double[] labels) { JavaRDD> trainRDD = - sc.parallelize(generateIsotonicInput(labels), 2).cache(); + jsc.parallelize(generateIsotonicInput(labels), 2).cache(); return new IsotonicRegression().run(trainRDD); } - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaLinearRegressionSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } - @Test public void testIsotonicRegressionJavaRDD() { IsotonicRegressionModel model = runIsotonicRegression(new double[]{1, 2, 3, 3, 1, 6, 7, 8, 11, 9, 10, 12}); Assert.assertArrayEquals( - new double[] {1, 2, 7.0/3, 7.0/3, 6, 7, 8, 10, 10, 12}, model.predictions(), 1.0e-14); + new double[]{1, 2, 7.0 / 3, 7.0 / 3, 6, 7, 8, 10, 10, 12}, model.predictions(), 1.0e-14); } @Test @@ -78,7 +63,7 @@ public void testIsotonicRegressionPredictionsJavaRDD() { IsotonicRegressionModel model = runIsotonicRegression(new double[]{1, 2, 3, 3, 1, 6, 7, 8, 11, 9, 10, 12}); - JavaDoubleRDD testRDD = sc.parallelizeDoubles(Arrays.asList(0.0, 1.0, 9.5, 12.0, 13.0)); + JavaDoubleRDD testRDD = jsc.parallelizeDoubles(Arrays.asList(0.0, 1.0, 9.5, 12.0, 13.0)); List predictions = model.predict(testRDD).collect(); Assert.assertEquals(1.0, predictions.get(0).doubleValue(), 1.0e-14); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java index 8950b48888b7..1458cc72bc17 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java @@ -17,35 +17,20 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.util.LinearDataGenerator; -public class JavaLassoSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaLassoSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaLassoSuite extends SharedSparkSession { int validatePrediction(List validationData, LassoModel model) { int numAccurate = 0; - for (LabeledPoint point: validationData) { + for (LabeledPoint point : validationData) { Double prediction = model.predict(point.features()); // A prediction is off if the prediction is more than 0.5 away from expected value. if (Math.abs(prediction - point.label()) <= 0.5) { @@ -61,15 +46,15 @@ public void runLassoUsingConstructor() { double A = 0.0; double[] weights = {-1.5, 1.0e-2}; - JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, - weights, nPoints, 42, 0.1), 2).cache(); + JavaRDD testRDD = jsc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, + weights, nPoints, 42, 0.1), 2).cache(); List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); LassoWithSGD lassoSGDImpl = new LassoWithSGD(); lassoSGDImpl.optimizer().setStepSize(1.0) - .setRegParam(0.01) - .setNumIterations(20); + .setRegParam(0.01) + .setNumIterations(20); LassoModel model = lassoSGDImpl.run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); @@ -82,10 +67,10 @@ public void runLassoUsingStaticMethods() { double A = 0.0; double[] weights = {-1.5, 1.0e-2}; - JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, - weights, nPoints, 42, 0.1), 2).cache(); + JavaRDD testRDD = jsc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, + weights, nPoints, 42, 0.1), 2).cache(); List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); LassoModel model = LassoWithSGD.train(testRDD.rdd(), 100, 1.0, 0.01, 1.0); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java index 24c4c20d9af1..a46b1321b3ca 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -17,42 +17,27 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.function.Function; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.util.LinearDataGenerator; -public class JavaLinearRegressionSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaLinearRegressionSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaLinearRegressionSuite extends SharedSparkSession { int validatePrediction(List validationData, LinearRegressionModel model) { int numAccurate = 0; - for (LabeledPoint point: validationData) { - Double prediction = model.predict(point.features()); - // A prediction is off if the prediction is more than 0.5 away from expected value. - if (Math.abs(prediction - point.label()) <= 0.5) { - numAccurate++; - } + for (LabeledPoint point : validationData) { + Double prediction = model.predict(point.features()); + // A prediction is off if the prediction is more than 0.5 away from expected value. + if (Math.abs(prediction - point.label()) <= 0.5) { + numAccurate++; + } } return numAccurate; } @@ -63,10 +48,10 @@ public void runLinearRegressionUsingConstructor() { double A = 3.0; double[] weights = {10, 10}; - JavaRDD testRDD = sc.parallelize( - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); + JavaRDD testRDD = jsc.parallelize( + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); linSGDImpl.setIntercept(true); @@ -82,10 +67,10 @@ public void runLinearRegressionUsingStaticMethods() { double A = 0.0; double[] weights = {10, 10}; - JavaRDD testRDD = sc.parallelize( - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); + JavaRDD testRDD = jsc.parallelize( + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); LinearRegressionModel model = LinearRegressionWithSGD.train(testRDD.rdd(), 100); @@ -98,7 +83,7 @@ public void testPredictJavaRDD() { int nPoints = 100; double A = 0.0; double[] weights = {10, 10}; - JavaRDD testRDD = sc.parallelize( + JavaRDD testRDD = jsc.parallelize( LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); LinearRegressionModel model = linSGDImpl.run(testRDD.rdd()); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java index c56db703ea0b..cb0097741234 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -17,37 +17,22 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.List; import java.util.Random; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.util.LinearDataGenerator; -public class JavaRidgeRegressionSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaRidgeRegressionSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaRidgeRegressionSuite extends SharedSparkSession { private static double predictionError(List validationData, RidgeRegressionModel model) { double errorSum = 0; - for (LabeledPoint point: validationData) { + for (LabeledPoint point : validationData) { Double prediction = model.predict(point.features()); errorSum += (prediction - point.label()) * (prediction - point.label()); } @@ -68,9 +53,9 @@ private static List generateRidgeData(int numPoints, int numFeatur public void runRidgeRegressionUsingConstructor() { int numExamples = 50; int numFeatures = 20; - List data = generateRidgeData(2*numExamples, numFeatures, 10.0); + List data = generateRidgeData(2 * numExamples, numFeatures, 10.0); - JavaRDD testRDD = sc.parallelize(data.subList(0, numExamples)); + JavaRDD testRDD = jsc.parallelize(data.subList(0, numExamples)); List validationData = data.subList(numExamples, 2 * numExamples); RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD(); @@ -94,7 +79,7 @@ public void runRidgeRegressionUsingStaticMethods() { int numFeatures = 20; List data = generateRidgeData(2 * numExamples, numFeatures, 10.0); - JavaRDD testRDD = sc.parallelize(data.subList(0, numExamples)); + JavaRDD testRDD = jsc.parallelize(data.subList(0, numExamples)); List validationData = data.subList(numExamples, 2 * numExamples); RidgeRegressionModel model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.0); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java index ea0ccd744898..ab554475d59a 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -36,7 +35,7 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext; import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStreamingLinearRegressionSuite implements Serializable { +public class JavaStreamingLinearRegressionSuite { protected transient JavaStreamingContext ssc; diff --git a/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java index 5f1d5987e809..1abaa39eadc2 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java @@ -17,20 +17,17 @@ package org.apache.spark.mllib.stat; -import java.io.Serializable; import java.util.Arrays; import java.util.List; import org.junit.After; import org.junit.Before; import org.junit.Test; - -import static org.apache.spark.streaming.JavaTestUtils.*; import static org.junit.Assert.assertEquals; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; @@ -38,36 +35,42 @@ import org.apache.spark.mllib.stat.test.ChiSqTestResult; import org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult; import org.apache.spark.mllib.stat.test.StreamingTest; +import org.apache.spark.sql.SparkSession; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; +import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStatisticsSuite implements Serializable { - private transient JavaSparkContext sc; +public class JavaStatisticsSuite { + private transient SparkSession spark; + private transient JavaSparkContext jsc; private transient JavaStreamingContext ssc; @Before public void setUp() { SparkConf conf = new SparkConf() - .setMaster("local[2]") - .setAppName("JavaStatistics") .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); - sc = new JavaSparkContext(conf); - ssc = new JavaStreamingContext(sc, new Duration(1000)); + spark = SparkSession.builder() + .master("local[2]") + .appName("JavaStatistics") + .config(conf) + .getOrCreate(); + jsc = new JavaSparkContext(spark.sparkContext()); + ssc = new JavaStreamingContext(jsc, new Duration(1000)); ssc.checkpoint("checkpoint"); } @After public void tearDown() { + spark.stop(); ssc.stop(); - ssc = null; - sc = null; + spark = null; } @Test public void testCorr() { - JavaRDD x = sc.parallelize(Arrays.asList(1.0, 2.0, 3.0, 4.0)); - JavaRDD y = sc.parallelize(Arrays.asList(1.1, 2.2, 3.1, 4.3)); + JavaRDD x = jsc.parallelize(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + JavaRDD y = jsc.parallelize(Arrays.asList(1.1, 2.2, 3.1, 4.3)); Double corr1 = Statistics.corr(x, y); Double corr2 = Statistics.corr(x, y, "pearson"); @@ -77,7 +80,7 @@ public void testCorr() { @Test public void kolmogorovSmirnovTest() { - JavaDoubleRDD data = sc.parallelizeDoubles(Arrays.asList(0.2, 1.0, -1.0, 2.0)); + JavaDoubleRDD data = jsc.parallelizeDoubles(Arrays.asList(0.2, 1.0, -1.0, 2.0)); KolmogorovSmirnovTestResult testResult1 = Statistics.kolmogorovSmirnovTest(data, "norm"); KolmogorovSmirnovTestResult testResult2 = Statistics.kolmogorovSmirnovTest( data, "norm", 0.0, 1.0); @@ -85,7 +88,7 @@ public void kolmogorovSmirnovTest() { @Test public void chiSqTest() { - JavaRDD data = sc.parallelize(Arrays.asList( + JavaRDD data = jsc.parallelize(Arrays.asList( new LabeledPoint(0.0, Vectors.dense(0.1, 2.3)), new LabeledPoint(1.0, Vectors.dense(1.5, 5.1)), new LabeledPoint(0.0, Vectors.dense(2.4, 8.1)))); diff --git a/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java b/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java index 8dd29061daaa..1dcbbcaa0223 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java @@ -17,41 +17,27 @@ package org.apache.spark.mllib.tree; -import java.io.Serializable; import java.util.HashMap; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.mllib.tree.configuration.Algo; import org.apache.spark.mllib.tree.configuration.Strategy; import org.apache.spark.mllib.tree.impurity.Gini; import org.apache.spark.mllib.tree.model.DecisionTreeModel; - -public class JavaDecisionTreeSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaDecisionTreeSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaDecisionTreeSuite extends SharedSparkSession { int validatePrediction(List validationData, DecisionTreeModel model) { int numCorrect = 0; - for (LabeledPoint point: validationData) { + for (LabeledPoint point : validationData) { Double prediction = model.predict(point.features()); if (prediction == point.label()) { numCorrect++; @@ -63,7 +49,7 @@ int validatePrediction(List validationData, DecisionTreeModel mode @Test public void runDTUsingConstructor() { List arr = DecisionTreeSuite.generateCategoricalDataPointsAsJavaList(); - JavaRDD rdd = sc.parallelize(arr); + JavaRDD rdd = jsc.parallelize(arr); HashMap categoricalFeaturesInfo = new HashMap<>(); categoricalFeaturesInfo.put(1, 2); // feature 1 has 2 categories @@ -71,7 +57,7 @@ public void runDTUsingConstructor() { int numClasses = 2; int maxBins = 100; Strategy strategy = new Strategy(Algo.Classification(), Gini.instance(), maxDepth, numClasses, - maxBins, categoricalFeaturesInfo); + maxBins, categoricalFeaturesInfo); DecisionTree learner = new DecisionTree(strategy); DecisionTreeModel model = learner.run(rdd.rdd()); @@ -83,7 +69,7 @@ public void runDTUsingConstructor() { @Test public void runDTUsingStaticMethods() { List arr = DecisionTreeSuite.generateCategoricalDataPointsAsJavaList(); - JavaRDD rdd = sc.parallelize(arr); + JavaRDD rdd = jsc.parallelize(arr); HashMap categoricalFeaturesInfo = new HashMap<>(); categoricalFeaturesInfo.put(1, 2); // feature 1 has 2 categories @@ -91,10 +77,18 @@ public void runDTUsingStaticMethods() { int numClasses = 2; int maxBins = 100; Strategy strategy = new Strategy(Algo.Classification(), Gini.instance(), maxDepth, numClasses, - maxBins, categoricalFeaturesInfo); + maxBins, categoricalFeaturesInfo); DecisionTreeModel model = DecisionTree$.MODULE$.train(rdd.rdd(), strategy); + // java compatibility test + JavaRDD predictions = model.predict(rdd.map(new Function() { + @Override + public Vector call(LabeledPoint v1) { + return v1.features(); + } + })); + int numCorrect = validatePrediction(arr, model); Assert.assertTrue(numCorrect == rdd.count()); } diff --git a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala index 1de638f24563..3b490cdf5601 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala @@ -27,9 +27,9 @@ import org.scalatest.mock.MockitoSugar.mock import org.apache.spark.SparkFunSuite import org.apache.spark.ml.Pipeline.SharedReadWrite import org.apache.spark.ml.feature.{HashingTF, MinMaxScaler} -import org.apache.spark.ml.param.{IntParam, ParamMap, ParamPair} +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.param.{IntParam, ParamMap} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types.StructType @@ -183,7 +183,7 @@ class PipelineSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul } test("pipeline validateParams") { - val df = sqlContext.createDataFrame( + val df = spark.createDataFrame( Seq( (1, Vectors.dense(0.0, 1.0, 4.0), 1.0), (2, Vectors.dense(1.0, 0.0, 4.0), 2.0), diff --git a/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala index dc91fc5f9e45..35586320cb82 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala @@ -18,10 +18,9 @@ package org.apache.spark.ml.ann import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ - class ANNSuite extends SparkFunSuite with MLlibTestSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/ml/ann/GradientSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/ann/GradientSuite.scala index 04cc426c40b5..f0c0183323c9 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/ann/GradientSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/ann/GradientSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.ann import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext class GradientSuite extends SparkFunSuite with MLlibTestSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala index 89afb94b0f96..4db5f03fb00b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala @@ -19,10 +19,10 @@ package org.apache.spark.ml.classification import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.classification.ClassifierSuite.MockClassifier +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset} @@ -32,7 +32,7 @@ class ClassifierSuite extends SparkFunSuite with MLlibTestSparkContext { test("extractLabeledPoints") { def getTestData(labels: Seq[Double]): DataFrame = { val data = labels.map { label: Double => LabeledPoint(label, Vectors.dense(0.0)) } - sqlContext.createDataFrame(data) + spark.createDataFrame(data) } val c = new MockClassifier @@ -72,7 +72,7 @@ class ClassifierSuite extends SparkFunSuite with MLlibTestSparkContext { test("getNumClasses") { def getTestData(labels: Seq[Double]): DataFrame = { val data = labels.map { label: Double => LabeledPoint(label, Vectors.dense(0.0)) } - sqlContext.createDataFrame(data) + spark.createDataFrame(data) } val c = new MockClassifier diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala index 29845b5554bf..089d30abb5ef 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.tree.{CategoricalSplit, InternalNode, LeafNode} import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree, DecisionTreeSuite => OldDecisionTreeSuite} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD @@ -44,17 +45,18 @@ class DecisionTreeClassifierSuite override def beforeAll() { super.beforeAll() categoricalDataPointsRDD = - sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()) + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()).map(_.asML) orderedLabeledPointsWithLabel0RDD = - sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel0()) + sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel0()).map(_.asML) orderedLabeledPointsWithLabel1RDD = - sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel1()) + sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel1()).map(_.asML) categoricalDataPointsForMulticlassRDD = - sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlass()) + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlass()).map(_.asML) continuousDataPointsForMulticlassRDD = - sc.parallelize(OldDecisionTreeSuite.generateContinuousDataPointsForMulticlass()) + sc.parallelize(OldDecisionTreeSuite.generateContinuousDataPointsForMulticlass()).map(_.asML) categoricalDataPointsForMulticlassForOrderedFeaturesRDD = sc.parallelize( OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures()) + .map(_.asML) } test("params") { @@ -337,13 +339,13 @@ class DecisionTreeClassifierSuite test("should support all NumericType labels and not support other types") { val dt = new DecisionTreeClassifier().setMaxDepth(1) MLTestingUtils.checkNumericTypes[DecisionTreeClassificationModel, DecisionTreeClassifier]( - dt, isClassification = true, sqlContext) { (expected, actual) => + dt, spark) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } test("Fitting without numClasses in metadata") { - val df: DataFrame = sqlContext.createDataFrame(TreeTests.featureImportanceData(sc)) + val df: DataFrame = spark.createDataFrame(TreeTests.featureImportanceData(sc)) val dt = new DecisionTreeClassifier().setMaxDepth(1) dt.fit(df) } @@ -395,7 +397,7 @@ private[ml] object DecisionTreeClassifierSuite extends SparkFunSuite { numClasses: Int): Unit = { val numFeatures = data.first().features.size val oldStrategy = dt.getOldStrategy(categoricalFeatures, numClasses) - val oldTree = OldDecisionTree.train(data, oldStrategy) + val oldTree = OldDecisionTree.train(data.map(OldLabeledPoint.fromML), oldStrategy) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses) val newTree = dt.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala index 087e201234b1..8d588ccfd354 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala @@ -18,13 +18,14 @@ package org.apache.spark.ml.classification import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree.LeafNode import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, GradientBoostedTrees => OldGBT} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -51,10 +52,13 @@ class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext override def beforeAll() { super.beforeAll() data = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100), 2) + .map(_.asML) trainData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 120), 2) + .map(_.asML) validationData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 80), 2) + .map(_.asML) } test("params") { @@ -106,7 +110,7 @@ class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext test("should support all NumericType labels and not support other types") { val gbt = new GBTClassifier().setMaxDepth(1) MLTestingUtils.checkNumericTypes[GBTClassificationModel, GBTClassifier]( - gbt, isClassification = true, sqlContext) { (expected, actual) => + gbt, spark) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } @@ -130,7 +134,7 @@ class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext */ test("Fitting without numClasses in metadata") { - val df: DataFrame = sqlContext.createDataFrame(TreeTests.featureImportanceData(sc)) + val df: DataFrame = spark.createDataFrame(TreeTests.featureImportanceData(sc)) val gbt = new GBTClassifier().setMaxDepth(1).setMaxIter(1) gbt.fit(df) } @@ -138,7 +142,7 @@ class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext test("extractLabeledPoints with bad data") { def getTestData(labels: Seq[Double]): DataFrame = { val data = labels.map { label: Double => LabeledPoint(label, Vectors.dense(0.0)) } - sqlContext.createDataFrame(data) + spark.createDataFrame(data) } val gbt = new GBTClassifier().setMaxDepth(1).setMaxIter(1) @@ -229,7 +233,7 @@ private object GBTClassifierSuite extends SparkFunSuite { val oldBoostingStrategy = gbt.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Classification) val oldGBT = new OldGBT(oldBoostingStrategy, gbt.getSeed.toInt) - val oldModel = oldGBT.run(data) + val oldModel = oldGBT.run(data.map(OldLabeledPoint.fromML)) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 2) val newModel = gbt.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 73e961dbbca2..a1b48539c46e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -17,18 +17,19 @@ package org.apache.spark.ml.classification +import scala.collection.JavaConverters._ import scala.language.existentials import scala.util.Random +import scala.util.control.Breaks._ import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.feature.{Instance, LabeledPoint} +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.classification.LogisticRegressionSuite._ -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.lit @@ -42,7 +43,7 @@ class LogisticRegressionSuite override def beforeAll(): Unit = { super.beforeAll() - dataset = sqlContext.createDataFrame(generateLogisticInput(1.0, 1.0, nPoints = 100, seed = 42)) + dataset = spark.createDataFrame(generateLogisticInput(1.0, 1.0, nPoints = 100, seed = 42)) binaryDataset = { val nPoints = 10000 @@ -54,7 +55,7 @@ class LogisticRegressionSuite generateMultinomialLogisticInput(coefficients, xMean, xVariance, addIntercept = true, nPoints, 42) - sqlContext.createDataFrame(sc.parallelize(testData, 4)) + spark.createDataFrame(sc.parallelize(testData, 4)) } } @@ -202,7 +203,7 @@ class LogisticRegressionSuite } test("logistic regression: Predictor, Classifier methods") { - val sqlContext = this.sqlContext + val spark = this.spark val lr = new LogisticRegression val model = lr.fit(dataset) @@ -256,6 +257,10 @@ class LogisticRegressionSuite assert(summarizer4.countInvalid === 2) assert(summarizer4.numClasses === 4) + val summarizer5 = new MultiClassSummarizer + assert(summarizer5.histogram.isEmpty) + assert(summarizer5.numClasses === 0) + // small map merges large one val summarizerA = summarizer1.merge(summarizer2) assert(summarizerA.hashCode() === summarizer2.hashCode()) @@ -864,8 +869,8 @@ class LogisticRegressionSuite } } - (sqlContext.createDataFrame(sc.parallelize(data1, 4)), - sqlContext.createDataFrame(sc.parallelize(data2, 4))) + (spark.createDataFrame(sc.parallelize(data1, 4)), + spark.createDataFrame(sc.parallelize(data2, 4))) } val trainer1a = (new LogisticRegression).setFitIntercept(true) @@ -938,7 +943,7 @@ class LogisticRegressionSuite test("should support all NumericType labels and not support other types") { val lr = new LogisticRegression().setMaxIter(1) MLTestingUtils.checkNumericTypes[LogisticRegressionModel, LogisticRegression]( - lr, isClassification = true, sqlContext) { (expected, actual) => + lr, spark) { (expected, actual) => assert(expected.intercept === actual.intercept) assert(expected.coefficients.toArray === actual.coefficients.toArray) } @@ -963,4 +968,122 @@ object LogisticRegressionSuite { "standardization" -> false, "threshold" -> 0.6 ) + + def generateLogisticInputAsList( + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): java.util.List[LabeledPoint] = { + generateLogisticInput(offset, scale, nPoints, seed).asJava + } + + // Generate input of the form Y = logistic(offset + scale*X) + def generateLogisticInput( + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + + val y = (0 until nPoints).map { i => + val p = 1.0 / (1.0 + math.exp(-(offset + scale * x1(i)))) + if (rnd.nextDouble() < p) 1.0 else 0.0 + } + + val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(Array(x1(i))))) + testData + } + + /** + * Generates `k` classes multinomial synthetic logistic input in `n` dimensional space given the + * model weights and mean/variance of the features. The synthetic data will be drawn from + * the probability distribution constructed by weights using the following formula. + * + * P(y = 0 | x) = 1 / norm + * P(y = 1 | x) = exp(x * w_1) / norm + * P(y = 2 | x) = exp(x * w_2) / norm + * ... + * P(y = k-1 | x) = exp(x * w_{k-1}) / norm + * where norm = 1 + exp(x * w_1) + exp(x * w_2) + ... + exp(x * w_{k-1}) + * + * @param weights matrix is flatten into a vector; as a result, the dimension of weights vector + * will be (k - 1) * (n + 1) if `addIntercept == true`, and + * if `addIntercept != true`, the dimension will be (k - 1) * n. + * @param xMean the mean of the generated features. Lots of time, if the features are not properly + * standardized, the algorithm with poor implementation will have difficulty + * to converge. + * @param xVariance the variance of the generated features. + * @param addIntercept whether to add intercept. + * @param nPoints the number of instance of generated data. + * @param seed the seed for random generator. For consistent testing result, it will be fixed. + */ + def generateMultinomialLogisticInput( + weights: Array[Double], + xMean: Array[Double], + xVariance: Array[Double], + addIntercept: Boolean, + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + + val xDim = xMean.length + val xWithInterceptsDim = if (addIntercept) xDim + 1 else xDim + val nClasses = weights.length / xWithInterceptsDim + 1 + + val x = Array.fill[Vector](nPoints)(Vectors.dense(Array.fill[Double](xDim)(rnd.nextGaussian()))) + + x.foreach { vector => + // This doesn't work if `vector` is a sparse vector. + val vectorArray = vector.toArray + var i = 0 + val len = vectorArray.length + while (i < len) { + vectorArray(i) = vectorArray(i) * math.sqrt(xVariance(i)) + xMean(i) + i += 1 + } + } + + val y = (0 until nPoints).map { idx => + val xArray = x(idx).toArray + val margins = Array.ofDim[Double](nClasses) + val probs = Array.ofDim[Double](nClasses) + + for (i <- 0 until nClasses - 1) { + for (j <- 0 until xDim) margins(i + 1) += weights(i * xWithInterceptsDim + j) * xArray(j) + if (addIntercept) margins(i + 1) += weights((i + 1) * xWithInterceptsDim - 1) + } + // Preventing the overflow when we compute the probability + val maxMargin = margins.max + if (maxMargin > 0) for (i <- 0 until nClasses) margins(i) -= maxMargin + + // Computing the probabilities for each class from the margins. + val norm = { + var temp = 0.0 + for (i <- 0 until nClasses) { + probs(i) = math.exp(margins(i)) + temp += probs(i) + } + temp + } + for (i <- 0 until nClasses) probs(i) /= norm + + // Compute the cumulative probability so we can generate a random number and assign a label. + for (i <- 1 until nClasses) probs(i) += probs(i - 1) + val p = rnd.nextDouble() + var y = 0 + breakable { + for (i <- 0 until nClasses) { + if (p < probs(i)) { + y = i + break + } + } + } + y + } + + val testData = (0 until nPoints).map(i => LabeledPoint(y(i), x(i))) + testData + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala index f41db31f1e7e..e809dd4092af 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala @@ -18,14 +18,16 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{Dataset, Row} class MultilayerPerceptronClassifierSuite @@ -36,7 +38,7 @@ class MultilayerPerceptronClassifierSuite override def beforeAll(): Unit = { super.beforeAll() - dataset = sqlContext.createDataFrame(Seq( + dataset = spark.createDataFrame(Seq( (Vectors.dense(0.0, 0.0), 0.0), (Vectors.dense(0.0, 1.0), 1.0), (Vectors.dense(1.0, 0.0), 1.0), @@ -68,6 +70,7 @@ class MultilayerPerceptronClassifierSuite .setBlockSize(1) .setSeed(123L) .setMaxIter(100) + .setSolver("l-bfgs") val model = trainer.fit(dataset) val result = model.transform(dataset) val predictionAndLabels = result.select("prediction", "label").collect() @@ -77,7 +80,7 @@ class MultilayerPerceptronClassifierSuite } test("Test setWeights by training restart") { - val dataFrame = sqlContext.createDataFrame(Seq( + val dataFrame = spark.createDataFrame(Seq( (Vectors.dense(0.0, 0.0), 0.0), (Vectors.dense(0.0, 1.0), 1.0), (Vectors.dense(1.0, 0.0), 1.0), @@ -91,9 +94,9 @@ class MultilayerPerceptronClassifierSuite .setMaxIter(1) .setTol(1e-6) val initialWeights = trainer.fit(dataFrame).weights - trainer.setWeights(initialWeights.copy) + trainer.setInitialWeights(initialWeights.copy) val weights1 = trainer.fit(dataFrame).weights - trainer.setWeights(initialWeights.copy) + trainer.setInitialWeights(initialWeights.copy) val weights2 = trainer.fit(dataFrame).weights assert(weights1 ~== weights2 absTol 10e-5, "Training should produce the same weights given equal initial weights and number of steps") @@ -113,7 +116,7 @@ class MultilayerPerceptronClassifierSuite // the input seed is somewhat magic, to make this test pass val rdd = sc.parallelize(generateMultinomialLogisticInput( coefficients, xMean, xVariance, true, nPoints, 1), 2) - val dataFrame = sqlContext.createDataFrame(rdd).toDF("label", "features") + val dataFrame = spark.createDataFrame(rdd).toDF("label", "features") val numClasses = 3 val numIterations = 100 val layers = Array[Int](4, 5, 4, numClasses) @@ -134,12 +137,13 @@ class MultilayerPerceptronClassifierSuite .setNumClasses(numClasses) lr.optimizer.setRegParam(0.0) .setNumIterations(numIterations) - val lrModel = lr.run(rdd) - val lrPredictionAndLabels = lrModel.predict(rdd.map(_.features)).zip(rdd.map(_.label)) + val lrModel = lr.run(rdd.map(OldLabeledPoint.fromML)) + val lrPredictionAndLabels = + lrModel.predict(rdd.map(p => OldVectors.fromML(p.features))).zip(rdd.map(_.label)) // MLP's predictions should not differ a lot from LR's. val lrMetrics = new MulticlassMetrics(lrPredictionAndLabels) val mlpMetrics = new MulticlassMetrics(mlpPredictionAndLabels) - assert(mlpMetrics.confusionMatrix ~== lrMetrics.confusionMatrix absTol 100) + assert(mlpMetrics.confusionMatrix.asML ~== lrMetrics.confusionMatrix.asML absTol 100) } test("read/write: MultilayerPerceptronClassifier") { @@ -169,7 +173,7 @@ class MultilayerPerceptronClassifierSuite val mpc = new MultilayerPerceptronClassifier().setLayers(layers).setMaxIter(1) MLTestingUtils.checkNumericTypes[ MultilayerPerceptronClassificationModel, MultilayerPerceptronClassifier]( - mpc, isClassification = true, sqlContext) { (expected, actual) => + mpc, spark) { (expected, actual) => assert(expected.layers === actual.layers) assert(expected.weights === actual.weights) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index 80a46fc70c75..ff52115ec0ec 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -17,16 +17,20 @@ package org.apache.spark.ml.classification -import breeze.linalg.{Vector => BV} +import scala.util.Random + +import breeze.linalg.{DenseVector => BDV, Vector => BV} +import breeze.stats.distributions.{Multinomial => BrzMultinomial} import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.classification.NaiveBayesSuite._ +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.classification.NaiveBayes.{Bernoulli, Multinomial} -import org.apache.spark.mllib.classification.NaiveBayesSuite._ -import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Dataset, Row} class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { @@ -43,7 +47,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext with Defa Array(0.10, 0.10, 0.70, 0.10) // label 2 ).map(_.map(math.log)) - dataset = sqlContext.createDataFrame(generateNaiveBayesInput(pi, theta, 100, 42)) + dataset = spark.createDataFrame(generateNaiveBayesInput(pi, theta, 100, 42)) } def validatePrediction(predictionAndLabels: DataFrame): Unit = { @@ -127,7 +131,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext with Defa val pi = Vectors.dense(piArray) val theta = new DenseMatrix(3, 4, thetaArray.flatten, true) - val testDataset = sqlContext.createDataFrame(generateNaiveBayesInput( + val testDataset = spark.createDataFrame(generateNaiveBayesInput( piArray, thetaArray, nPoints, 42, "multinomial")) val nb = new NaiveBayes().setSmoothing(1.0).setModelType("multinomial") val model = nb.fit(testDataset) @@ -135,7 +139,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext with Defa validateModelFit(pi, theta, model) assert(model.hasParent) - val validationDataset = sqlContext.createDataFrame(generateNaiveBayesInput( + val validationDataset = spark.createDataFrame(generateNaiveBayesInput( piArray, thetaArray, nPoints, 17, "multinomial")) val predictionAndLabels = model.transform(validationDataset).select("prediction", "label") @@ -157,7 +161,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext with Defa val pi = Vectors.dense(piArray) val theta = new DenseMatrix(3, 12, thetaArray.flatten, true) - val testDataset = sqlContext.createDataFrame(generateNaiveBayesInput( + val testDataset = spark.createDataFrame(generateNaiveBayesInput( piArray, thetaArray, nPoints, 45, "bernoulli")) val nb = new NaiveBayes().setSmoothing(1.0).setModelType("bernoulli") val model = nb.fit(testDataset) @@ -165,7 +169,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext with Defa validateModelFit(pi, theta, model) assert(model.hasParent) - val validationDataset = sqlContext.createDataFrame(generateNaiveBayesInput( + val validationDataset = spark.createDataFrame(generateNaiveBayesInput( piArray, thetaArray, nPoints, 20, "bernoulli")) val predictionAndLabels = model.transform(validationDataset).select("prediction", "label") @@ -188,7 +192,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext with Defa test("should support all NumericType labels and not support other types") { val nb = new NaiveBayes() MLTestingUtils.checkNumericTypes[NaiveBayesModel, NaiveBayes]( - nb, isClassification = true, sqlContext) { (expected, actual) => + nb, spark) { (expected, actual) => assert(expected.pi === actual.pi) assert(expected.theta === actual.theta) } @@ -206,4 +210,48 @@ object NaiveBayesSuite { "predictionCol" -> "myPrediction", "smoothing" -> 0.1 ) + + private def calcLabel(p: Double, pi: Array[Double]): Int = { + var sum = 0.0 + for (j <- 0 until pi.length) { + sum += pi(j) + if (p < sum) return j + } + -1 + } + + // Generate input of the form Y = (theta * x).argmax() + def generateNaiveBayesInput( + pi: Array[Double], // 1XC + theta: Array[Array[Double]], // CXD + nPoints: Int, + seed: Int, + modelType: String = Multinomial, + sample: Int = 10): Seq[LabeledPoint] = { + val D = theta(0).length + val rnd = new Random(seed) + val _pi = pi.map(math.pow(math.E, _)) + val _theta = theta.map(row => row.map(math.pow(math.E, _))) + + for (i <- 0 until nPoints) yield { + val y = calcLabel(rnd.nextDouble(), _pi) + val xi = modelType match { + case Bernoulli => Array.tabulate[Double] (D) { j => + if (rnd.nextDouble () < _theta(y)(j) ) 1 else 0 + } + case Multinomial => + val mult = BrzMultinomial(BDV(_theta(y))) + val emptyMap = (0 until D).map(x => (x, 0.0)).toMap + val counts = emptyMap ++ mult.sample(sample).groupBy(x => x).map { + case (index, reps) => (index, reps.size.toDouble) + } + counts.toArray.sortBy(_._1).map(_._2) + case _ => + // This should never happen. + throw new UnknownError(s"Invalid modelType: $modelType.") + } + + LabeledPoint(y, Vectors.dense(xi)) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala index 51871a9babe4..361dd74cb082 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala @@ -19,14 +19,16 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.NominalAttribute +import org.apache.spark.ml.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.feature.StringIndexer +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.{ParamMap, ParamsSuite} import org.apache.spark.ml.util.{DefaultReadWriteTest, MetadataUtils, MLTestingUtils} -import org.apache.spark.mllib.classification.LogisticRegressionSuite._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD @@ -53,7 +55,7 @@ class OneVsRestSuite extends SparkFunSuite with MLlibTestSparkContext with Defau val xVariance = Array(0.6856, 0.1899, 3.116, 0.581) rdd = sc.parallelize(generateMultinomialLogisticInput( coefficients, xMean, xVariance, true, nPoints, 42), 2) - dataset = sqlContext.createDataFrame(rdd) + dataset = spark.createDataFrame(rdd) } test("params") { @@ -88,8 +90,8 @@ class OneVsRestSuite extends SparkFunSuite with MLlibTestSparkContext with Defau val lr = new LogisticRegressionWithLBFGS().setIntercept(true).setNumClasses(numClasses) lr.optimizer.setRegParam(0.1).setNumIterations(100) - val model = lr.run(rdd) - val results = model.predict(rdd.map(_.features)).zip(rdd.map(_.label)) + val model = lr.run(rdd.map(OldLabeledPoint.fromML)) + val results = model.predict(rdd.map(p => OldVectors.fromML(p.features))).zip(rdd.map(_.label)) // determine the #confusion matrix in each class. // bound how much error we allow compared to multinomial logistic regression. val expectedMetrics = new MulticlassMetrics(results) @@ -228,7 +230,7 @@ class OneVsRestSuite extends SparkFunSuite with MLlibTestSparkContext with Defau test("should support all NumericType labels and not support other types") { val ovr = new OneVsRest().setClassifier(new LogisticRegression().setMaxIter(1)) MLTestingUtils.checkNumericTypes[OneVsRestModel, OneVsRest]( - ovr, isClassification = true, sqlContext) { (expected, actual) => + ovr, spark) { (expected, actual) => val expectedModels = expected.models.map(m => m.asInstanceOf[LogisticRegressionModel]) val actualModels = actual.models.map(m => m.asInstanceOf[LogisticRegressionModel]) assert(expectedModels.length === actualModels.length) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala index cfa75ecf387c..b3bd2b3e57b3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.{Vector, Vectors} final class TestProbabilisticClassificationModel( override val uid: String, diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala index 90744353d9d3..2e99ee157ae9 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.tree.LeafNode import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, RandomForest => OldRandomForest} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -46,8 +47,10 @@ class RandomForestClassifierSuite super.beforeAll() orderedLabeledPoints50_1000 = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)) + .map(_.asML) orderedLabeledPoints5_20 = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 5, 20)) + .map(_.asML) } ///////////////////////////////////////////////////////////////////////////// @@ -155,7 +158,7 @@ class RandomForestClassifierSuite } test("Fitting without numClasses in metadata") { - val df: DataFrame = sqlContext.createDataFrame(TreeTests.featureImportanceData(sc)) + val df: DataFrame = spark.createDataFrame(TreeTests.featureImportanceData(sc)) val rf = new RandomForestClassifier().setMaxDepth(1).setNumTrees(1) rf.fit(df) } @@ -189,7 +192,7 @@ class RandomForestClassifierSuite test("should support all NumericType labels and not support other types") { val rf = new RandomForestClassifier().setMaxDepth(1) MLTestingUtils.checkNumericTypes[RandomForestClassificationModel, RandomForestClassifier]( - rf, isClassification = true, sqlContext) { (expected, actual) => + rf, spark) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } @@ -233,7 +236,8 @@ private object RandomForestClassifierSuite extends SparkFunSuite { val oldStrategy = rf.getOldStrategy(categoricalFeatures, numClasses, OldAlgo.Classification, rf.getOldImpurity) val oldModel = OldRandomForest.trainClassifier( - data, oldStrategy, rf.getNumTrees, rf.getFeatureSubsetStrategy, rf.getSeed.toInt) + data.map(OldLabeledPoint.fromML), oldStrategy, rf.getNumTrees, rf.getFeatureSubsetStrategy, + rf.getSeed.toInt) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses) val newModel = rf.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala index 212ea7a0a960..4f7d4418a8d0 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala @@ -30,7 +30,7 @@ class BisectingKMeansSuite override def beforeAll(): Unit = { super.beforeAll() - dataset = KMeansSuite.generateKMeansData(sqlContext, 50, 3, k) + dataset = KMeansSuite.generateKMeansData(spark, 50, 3, k) } test("default parameters") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala index 9d868174c172..04366f525028 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala @@ -32,7 +32,7 @@ class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext override def beforeAll(): Unit = { super.beforeAll() - dataset = KMeansSuite.generateKMeansData(sqlContext, 50, 3, k) + dataset = KMeansSuite.generateKMeansData(spark, 50, 3, k) } test("default parameters") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index 2ca386e4229c..88f31a1cd26f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -18,11 +18,11 @@ package org.apache.spark.ml.clustering import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.clustering.{KMeans => MLlibKMeans} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{DataFrame, Dataset, SQLContext} +import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} private[clustering] case class TestRow(features: Vector) @@ -34,7 +34,7 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultR override def beforeAll(): Unit = { super.beforeAll() - dataset = KMeansSuite.generateKMeansData(sqlContext, 50, 3, k) + dataset = KMeansSuite.generateKMeansData(spark, 50, 3, k) } test("default parameters") { @@ -117,6 +117,21 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultR assert(clusterSizes.forall(_ >= 0)) } + test("KMeansModel transform with non-default feature and prediction cols") { + val featuresColName = "kmeans_model_features" + val predictionColName = "kmeans_model_prediction" + + val model = new KMeans().setK(k).setSeed(1).fit(dataset) + model.setFeaturesCol(featuresColName).setPredictionCol(predictionColName) + + val transformed = model.transform(dataset.withColumnRenamed("features", featuresColName)) + Seq(featuresColName, predictionColName).foreach { column => + assert(transformed.columns.contains(column)) + } + assert(model.getFeaturesCol == featuresColName) + assert(model.getPredictionCol == predictionColName) + } + test("read/write") { def checkModelData(model: KMeansModel, model2: KMeansModel): Unit = { assert(model.clusterCenters === model2.clusterCenters) @@ -127,11 +142,11 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultR } object KMeansSuite { - def generateKMeansData(sql: SQLContext, rows: Int, dim: Int, k: Int): DataFrame = { - val sc = sql.sparkContext + def generateKMeansData(spark: SparkSession, rows: Int, dim: Int, k: Int): DataFrame = { + val sc = spark.sparkContext val rdd = sc.parallelize(1 to rows).map(i => Vectors.dense(Array.fill(dim)((i % k).toDouble))) .map(v => new TestRow(v)) - sql.createDataFrame(rdd) + spark.createDataFrame(rdd) } /** diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala index 6cb07aecb952..ddfa87555427 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala @@ -17,30 +17,30 @@ package org.apache.spark.ml.clustering -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ -import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext} +import org.apache.spark.sql._ object LDASuite { def generateLDAData( - sql: SQLContext, + spark: SparkSession, rows: Int, k: Int, vocabSize: Int): DataFrame = { val avgWC = 1 // average instances of each word in a doc - val sc = sql.sparkContext + val sc = spark.sparkContext val rng = new java.util.Random() rng.setSeed(1) val rdd = sc.parallelize(1 to rows).map { i => Vectors.dense(Array.fill(vocabSize)(rng.nextInt(2 * avgWC).toDouble)) }.map(v => new TestRow(v)) - sql.createDataFrame(rdd) + spark.createDataFrame(rdd) } /** @@ -68,7 +68,7 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext with DefaultRead override def beforeAll(): Unit = { super.beforeAll() - dataset = LDASuite.generateLDAData(sqlContext, 50, k, vocabSize) + dataset = LDASuite.generateLDAData(spark, 50, k, vocabSize) } test("default parameters") { @@ -140,7 +140,7 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext with DefaultRead new LDA().setTopicConcentration(-1.1) } - val dummyDF = sqlContext.createDataFrame(Seq( + val dummyDF = spark.createDataFrame(Seq( (1, Vectors.dense(1.0, 2.0)))).toDF("id", "features") // validate parameters lda.transformSchema(dummyDF.schema) @@ -274,7 +274,7 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext with DefaultRead // There should be 1 checkpoint remaining. assert(model.getCheckpointFiles.length === 1) val checkpointFile = new Path(model.getCheckpointFiles.head) - val fs = checkpointFile.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + val fs = checkpointFile.getFileSystem(spark.sparkContext.hadoopConfiguration) assert(fs.exists(checkpointFile)) model.deleteCheckpointFiles() assert(model.getCheckpointFiles.isEmpty) diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala index ff3452217808..9ee3df5eb5e3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.ml.evaluation import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext class BinaryClassificationEvaluatorSuite @@ -42,21 +42,21 @@ class BinaryClassificationEvaluatorSuite val evaluator = new BinaryClassificationEvaluator() .setMetricName("areaUnderPR") - val vectorDF = sqlContext.createDataFrame(Seq( + val vectorDF = spark.createDataFrame(Seq( (0d, Vectors.dense(12, 2.5)), (1d, Vectors.dense(1, 3)), (0d, Vectors.dense(10, 2)) )).toDF("label", "rawPrediction") assert(evaluator.evaluate(vectorDF) === 1.0) - val doubleDF = sqlContext.createDataFrame(Seq( + val doubleDF = spark.createDataFrame(Seq( (0d, 0d), (1d, 1d), (0d, 0d) )).toDF("label", "rawPrediction") assert(evaluator.evaluate(doubleDF) === 1.0) - val stringDF = sqlContext.createDataFrame(Seq( + val stringDF = spark.createDataFrame(Seq( (0d, "0d"), (1d, "1d"), (0d, "0d") @@ -71,6 +71,6 @@ class BinaryClassificationEvaluatorSuite test("should support all NumericType labels and not support other types") { val evaluator = new BinaryClassificationEvaluator().setRawPredictionCol("prediction") - MLTestingUtils.checkNumericTypes(evaluator, sqlContext) + MLTestingUtils.checkNumericTypes(evaluator, spark) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala index 87e511a368a2..522f6675d7f4 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala @@ -38,6 +38,6 @@ class MulticlassClassificationEvaluatorSuite } test("should support all NumericType labels and not support other types") { - MLTestingUtils.checkNumericTypes(new MulticlassClassificationEvaluator, sqlContext) + MLTestingUtils.checkNumericTypes(new MulticlassClassificationEvaluator, spark) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala index c7b94830696c..42ff8adf6bd6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala @@ -42,9 +42,9 @@ class RegressionEvaluatorSuite * data.map(x=> x.label + ", " + x.features(0) + ", " + x.features(1)) * .saveAsTextFile("path") */ - val dataset = sqlContext.createDataFrame( + val dataset = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2)) + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2).map(_.asML)) /** * Using the following R code to load the data, train the model and evaluate metrics. @@ -85,6 +85,6 @@ class RegressionEvaluatorSuite } test("should support all NumericType labels and not support other types") { - MLTestingUtils.checkNumericTypes(new RegressionEvaluator, sqlContext) + MLTestingUtils.checkNumericTypes(new RegressionEvaluator, spark) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala index 714b9db3aa19..9cb84a6ee9b8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Row} @@ -39,7 +39,7 @@ class BinarizerSuite extends SparkFunSuite with MLlibTestSparkContext with Defau test("Binarize continuous features with default parameter") { val defaultBinarized: Array[Double] = data.map(x => if (x > 0.0) 1.0 else 0.0) - val dataFrame: DataFrame = sqlContext.createDataFrame( + val dataFrame: DataFrame = spark.createDataFrame( data.zip(defaultBinarized)).toDF("feature", "expected") val binarizer: Binarizer = new Binarizer() @@ -55,7 +55,7 @@ class BinarizerSuite extends SparkFunSuite with MLlibTestSparkContext with Defau test("Binarize continuous features with setter") { val threshold: Double = 0.2 val thresholdBinarized: Array[Double] = data.map(x => if (x > threshold) 1.0 else 0.0) - val dataFrame: DataFrame = sqlContext.createDataFrame( + val dataFrame: DataFrame = spark.createDataFrame( data.zip(thresholdBinarized)).toDF("feature", "expected") val binarizer: Binarizer = new Binarizer() @@ -71,7 +71,7 @@ class BinarizerSuite extends SparkFunSuite with MLlibTestSparkContext with Defau test("Binarize vector of continuous features with default parameter") { val defaultBinarized: Array[Double] = data.map(x => if (x > 0.0) 1.0 else 0.0) - val dataFrame: DataFrame = sqlContext.createDataFrame(Seq( + val dataFrame: DataFrame = spark.createDataFrame(Seq( (Vectors.dense(data), Vectors.dense(defaultBinarized)) )).toDF("feature", "expected") @@ -88,7 +88,7 @@ class BinarizerSuite extends SparkFunSuite with MLlibTestSparkContext with Defau test("Binarize vector of continuous features with setter") { val threshold: Double = 0.2 val defaultBinarized: Array[Double] = data.map(x => if (x > threshold) 1.0 else 0.0) - val dataFrame: DataFrame = sqlContext.createDataFrame(Seq( + val dataFrame: DataFrame = spark.createDataFrame(Seq( (Vectors.dense(data), Vectors.dense(defaultBinarized)) )).toDF("feature", "expected") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala index 9ea7d431763a..cd10c78311e1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala @@ -20,11 +20,11 @@ package org.apache.spark.ml.feature import scala.util.Random import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} class BucketizerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { @@ -39,7 +39,7 @@ class BucketizerSuite extends SparkFunSuite with MLlibTestSparkContext with Defa val validData = Array(-0.5, -0.3, 0.0, 0.2) val expectedBuckets = Array(0.0, 0.0, 1.0, 1.0) val dataFrame: DataFrame = - sqlContext.createDataFrame(validData.zip(expectedBuckets)).toDF("feature", "expected") + spark.createDataFrame(validData.zip(expectedBuckets)).toDF("feature", "expected") val bucketizer: Bucketizer = new Bucketizer() .setInputCol("feature") @@ -55,13 +55,13 @@ class BucketizerSuite extends SparkFunSuite with MLlibTestSparkContext with Defa // Check for exceptions when using a set of invalid feature values. val invalidData1: Array[Double] = Array(-0.9) ++ validData val invalidData2 = Array(0.51) ++ validData - val badDF1 = sqlContext.createDataFrame(invalidData1.zipWithIndex).toDF("feature", "idx") + val badDF1 = spark.createDataFrame(invalidData1.zipWithIndex).toDF("feature", "idx") withClue("Invalid feature value -0.9 was not caught as an invalid feature!") { intercept[SparkException] { bucketizer.transform(badDF1).collect() } } - val badDF2 = sqlContext.createDataFrame(invalidData2.zipWithIndex).toDF("feature", "idx") + val badDF2 = spark.createDataFrame(invalidData2.zipWithIndex).toDF("feature", "idx") withClue("Invalid feature value 0.51 was not caught as an invalid feature!") { intercept[SparkException] { bucketizer.transform(badDF2).collect() @@ -74,7 +74,7 @@ class BucketizerSuite extends SparkFunSuite with MLlibTestSparkContext with Defa val validData = Array(-0.9, -0.5, -0.3, 0.0, 0.2, 0.5, 0.9) val expectedBuckets = Array(0.0, 1.0, 1.0, 2.0, 2.0, 3.0, 3.0) val dataFrame: DataFrame = - sqlContext.createDataFrame(validData.zip(expectedBuckets)).toDF("feature", "expected") + spark.createDataFrame(validData.zip(expectedBuckets)).toDF("feature", "expected") val bucketizer: Bucketizer = new Bucketizer() .setInputCol("feature") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala index 7827db2794cf..40d5b4881f83 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala @@ -18,20 +18,22 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { test("Test Chi-Square selector") { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder + .master("local[2]") + .appName("ChiSqSelectorSuite") + .getOrCreate() + import spark.implicits._ val data = Seq( LabeledPoint(0.0, Vectors.sparse(3, Array((0, 8.0), (1, 7.0)))), @@ -78,4 +80,12 @@ class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext val newInstance = testDefaultReadWrite(instance) assert(newInstance.selectedFeatures === instance.selectedFeatures) } + + test("should support all NumericType labels and not support other types") { + val css = new ChiSqSelector() + MLTestingUtils.checkNumericTypes[ChiSqSelectorModel, ChiSqSelector]( + css, spark) { (expected, actual) => + assert(expected.selectedFeatures === actual.selectedFeatures) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala index 7641e3b8cf66..a59203c33d81 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext @@ -35,7 +35,7 @@ class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext private def split(s: String): Seq[String] = s.split("\\s+") test("CountVectorizerModel common cases") { - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, split("a b c d"), Vectors.sparse(4, Seq((0, 1.0), (1, 1.0), (2, 1.0), (3, 1.0)))), (1, split("a b b c d a"), @@ -55,7 +55,7 @@ class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext } test("CountVectorizer common cases") { - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, split("a b c d e"), Vectors.sparse(5, Seq((0, 1.0), (1, 1.0), (2, 1.0), (3, 1.0), (4, 1.0)))), (1, split("a a a a a a"), Vectors.sparse(5, Seq((0, 6.0)))), @@ -76,7 +76,7 @@ class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext } test("CountVectorizer vocabSize and minDF") { - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, split("a b c d"), Vectors.sparse(3, Seq((0, 1.0), (1, 1.0)))), (1, split("a b c"), Vectors.sparse(3, Seq((0, 1.0), (1, 1.0)))), (2, split("a b"), Vectors.sparse(3, Seq((0, 1.0), (1, 1.0)))), @@ -118,7 +118,7 @@ class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext test("CountVectorizer throws exception when vocab is empty") { intercept[IllegalArgumentException] { - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, split("a a b b c c")), (1, split("aa bb cc"))) ).toDF("id", "words") @@ -132,7 +132,7 @@ class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext } test("CountVectorizerModel with minTF count") { - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, split("a a a b b c c c d "), Vectors.sparse(4, Seq((0, 3.0), (2, 3.0)))), (1, split("c c c c c c"), Vectors.sparse(4, Seq((2, 6.0)))), (2, split("a"), Vectors.sparse(4, Seq())), @@ -151,7 +151,7 @@ class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext } test("CountVectorizerModel with minTF freq") { - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, split("a a a b b c c c d "), Vectors.sparse(4, Seq((0, 3.0), (2, 3.0)))), (1, split("c c c c c c"), Vectors.sparse(4, Seq((2, 6.0)))), (2, split("a"), Vectors.sparse(4, Seq((0, 1.0)))), @@ -170,7 +170,7 @@ class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext } test("CountVectorizerModel and CountVectorizer with binary") { - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, split("a a a a b b b b c d"), Vectors.sparse(4, Seq((0, 1.0), (1, 1.0), (2, 1.0), (3, 1.0)))), (1, split("c c c"), Vectors.sparse(4, Seq((2, 1.0)))), diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala index 36cafa290f08..c02e9610418b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala @@ -22,8 +22,8 @@ import scala.beans.BeanInfo import edu.emory.mathcs.jtransforms.dct.DoubleDCT_1D import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.Row @@ -63,7 +63,7 @@ class DCTSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultRead } val expectedResult = Vectors.dense(expectedResultBuffer) - val dataset = sqlContext.createDataFrame(Seq( + val dataset = spark.createDataFrame(Seq( DCTTestData(data, expectedResult) )) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala index fc1c05de233e..a4cca27be781 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext class ElementwiseProductSuite diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala index 44bad4aba4dd..99b800776bb6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala @@ -19,12 +19,12 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.feature.{HashingTF => MLlibHashingTF} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.Utils class HashingTFSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { @@ -34,7 +34,7 @@ class HashingTFSuite extends SparkFunSuite with MLlibTestSparkContext with Defau } test("hashingTF") { - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, "a a b b c d".split(" ").toSeq) )).toDF("id", "words") val n = 100 @@ -54,7 +54,7 @@ class HashingTFSuite extends SparkFunSuite with MLlibTestSparkContext with Defau } test("applying binary term freqs") { - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, "a a b c c c".split(" ").toSeq) )).toDF("id", "words") val n = 100 diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala index bc958c15857b..09dc8b9b932f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.feature.{IDFModel => OldIDFModel} -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row class IDFSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { @@ -60,7 +61,7 @@ class IDFSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultRead }) val expected = scaleDataWithIDF(data, idf) - val df = sqlContext.createDataFrame(data.zip(expected)).toDF("features", "expected") + val df = spark.createDataFrame(data.zip(expected)).toDF("features", "expected") val idfModel = new IDF() .setInputCol("features") @@ -86,7 +87,7 @@ class IDFSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultRead }) val expected = scaleDataWithIDF(data, idf) - val df = sqlContext.createDataFrame(data.zip(expected)).toDF("features", "expected") + val df = spark.createDataFrame(data.zip(expected)).toDF("features", "expected") val idfModel = new IDF() .setInputCol("features") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala index 0d4e00668ddb..3429172a8c90 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala @@ -21,9 +21,9 @@ import scala.collection.mutable.ArrayBuilder import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.functions.col @@ -59,7 +59,7 @@ class InteractionSuite extends SparkFunSuite with MLlibTestSparkContext with Def } test("numeric interaction") { - val data = sqlContext.createDataFrame( + val data = spark.createDataFrame( Seq( (2, Vectors.dense(3.0, 4.0)), (1, Vectors.dense(1.0, 5.0))) @@ -74,7 +74,7 @@ class InteractionSuite extends SparkFunSuite with MLlibTestSparkContext with Def col("b").as("b", groupAttr.toMetadata())) val trans = new Interaction().setInputCols(Array("a", "b")).setOutputCol("features") val res = trans.transform(df) - val expected = sqlContext.createDataFrame( + val expected = spark.createDataFrame( Seq( (2, Vectors.dense(3.0, 4.0), Vectors.dense(6.0, 8.0)), (1, Vectors.dense(1.0, 5.0), Vectors.dense(1.0, 5.0))) @@ -90,7 +90,7 @@ class InteractionSuite extends SparkFunSuite with MLlibTestSparkContext with Def } test("nominal interaction") { - val data = sqlContext.createDataFrame( + val data = spark.createDataFrame( Seq( (2, Vectors.dense(3.0, 4.0)), (1, Vectors.dense(1.0, 5.0))) @@ -106,7 +106,7 @@ class InteractionSuite extends SparkFunSuite with MLlibTestSparkContext with Def col("b").as("b", groupAttr.toMetadata())) val trans = new Interaction().setInputCols(Array("a", "b")).setOutputCol("features") val res = trans.transform(df) - val expected = sqlContext.createDataFrame( + val expected = spark.createDataFrame( Seq( (2, Vectors.dense(3.0, 4.0), Vectors.dense(0, 0, 0, 0, 3, 4)), (1, Vectors.dense(1.0, 5.0), Vectors.dense(0, 0, 1, 5, 0, 0))) @@ -126,7 +126,7 @@ class InteractionSuite extends SparkFunSuite with MLlibTestSparkContext with Def } test("default attr names") { - val data = sqlContext.createDataFrame( + val data = spark.createDataFrame( Seq( (2, Vectors.dense(0.0, 4.0), 1.0), (1, Vectors.dense(1.0, 5.0), 10.0)) @@ -142,7 +142,7 @@ class InteractionSuite extends SparkFunSuite with MLlibTestSparkContext with Def col("c").as("c", NumericAttribute.defaultAttr.toMetadata())) val trans = new Interaction().setInputCols(Array("a", "b", "c")).setOutputCol("features") val res = trans.transform(df) - val expected = sqlContext.createDataFrame( + val expected = spark.createDataFrame( Seq( (2, Vectors.dense(0.0, 4.0), 1.0, Vectors.dense(0, 0, 0, 0, 0, 0, 1, 0, 4)), (1, Vectors.dense(1.0, 5.0), 10.0, Vectors.dense(0, 0, 0, 0, 10, 50, 0, 0, 0))) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala index e083d4713680..d6400ee02f95 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.Row @@ -36,7 +36,7 @@ class MaxAbsScalerSuite extends SparkFunSuite with MLlibTestSparkContext with De Vectors.sparse(3, Array(0, 2), Array(-1, -1)), Vectors.sparse(3, Array(0), Array(-0.75))) - val df = sqlContext.createDataFrame(data.zip(expected)).toDF("features", "expected") + val df = spark.createDataFrame(data.zip(expected)).toDF("features", "expected") val scaler = new MaxAbsScaler() .setInputCol("features") .setOutputCol("scaled") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala index 87206c777e35..5da84711758c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.Row @@ -38,7 +38,7 @@ class MinMaxScalerSuite extends SparkFunSuite with MLlibTestSparkContext with De Vectors.sparse(3, Array(0, 2), Array(5, 5)), Vectors.sparse(3, Array(0), Array(-2.5))) - val df = sqlContext.createDataFrame(data.zip(expected)).toDF("features", "expected") + val df = spark.createDataFrame(data.zip(expected)).toDF("features", "expected") val scaler = new MinMaxScaler() .setInputCol("features") .setOutputCol("scaled") @@ -57,7 +57,7 @@ class MinMaxScalerSuite extends SparkFunSuite with MLlibTestSparkContext with De test("MinMaxScaler arguments max must be larger than min") { withClue("arguments max must be larger than min") { - val dummyDF = sqlContext.createDataFrame(Seq( + val dummyDF = spark.createDataFrame(Seq( (1, Vectors.dense(1.0, 2.0)))).toDF("id", "feature") intercept[IllegalArgumentException] { val scaler = new MinMaxScaler().setMin(10).setMax(0).setInputCol("feature") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala index a9421e682519..e5288d9259d3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala @@ -34,7 +34,7 @@ class NGramSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultRe val nGram = new NGram() .setInputCol("inputTokens") .setOutputCol("nGrams") - val dataset = sqlContext.createDataFrame(Seq( + val dataset = spark.createDataFrame(Seq( NGramTestData( Array("Test", "for", "ngram", "."), Array("Test for", "for ngram", "ngram .") @@ -47,7 +47,7 @@ class NGramSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultRe .setInputCol("inputTokens") .setOutputCol("nGrams") .setN(4) - val dataset = sqlContext.createDataFrame(Seq( + val dataset = spark.createDataFrame(Seq( NGramTestData( Array("a", "b", "c", "d", "e"), Array("a b c d", "b c d e") @@ -60,7 +60,7 @@ class NGramSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultRe .setInputCol("inputTokens") .setOutputCol("nGrams") .setN(4) - val dataset = sqlContext.createDataFrame(Seq( + val dataset = spark.createDataFrame(Seq( NGramTestData( Array(), Array() @@ -73,7 +73,7 @@ class NGramSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultRe .setInputCol("inputTokens") .setOutputCol("nGrams") .setN(6) - val dataset = sqlContext.createDataFrame(Seq( + val dataset = spark.createDataFrame(Seq( NGramTestData( Array("a", "b", "c", "d", "e"), Array() diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala index 468833901995..b69283171446 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala @@ -18,10 +18,10 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} @@ -61,7 +61,7 @@ class NormalizerSuite extends SparkFunSuite with MLlibTestSparkContext with Defa Vectors.sparse(3, Seq()) ) - dataFrame = sqlContext.createDataFrame(sc.parallelize(data, 2).map(NormalizerSuite.FeatureData)) + dataFrame = spark.createDataFrame(sc.parallelize(data, 2).map(NormalizerSuite.FeatureData)) normalizer = new Normalizer() .setInputCol("features") .setOutputCol("normalized_features") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala index 49803aef7158..7841b4fbc77a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.{AttributeGroup, BinaryAttribute, NominalAttribute} +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions.col @@ -32,7 +32,7 @@ class OneHotEncoderSuite def stringIndexed(): DataFrame = { val data = sc.parallelize(Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")), 2) - val df = sqlContext.createDataFrame(data).toDF("id", "label") + val df = spark.createDataFrame(data).toDF("id", "label") val indexer = new StringIndexer() .setInputCol("label") .setOutputCol("labelIndex") @@ -81,7 +81,7 @@ class OneHotEncoderSuite test("input column with ML attribute") { val attr = NominalAttribute.defaultAttr.withValues("small", "medium", "large") - val df = sqlContext.createDataFrame(Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply)).toDF("size") + val df = spark.createDataFrame(Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply)).toDF("size") .select(col("size").as("size", attr.toMetadata())) val encoder = new OneHotEncoder() .setInputCol("size") @@ -94,7 +94,7 @@ class OneHotEncoderSuite } test("input column without ML attribute") { - val df = sqlContext.createDataFrame(Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply)).toDF("index") + val df = spark.createDataFrame(Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply)).toDF("index") val encoder = new OneHotEncoder() .setInputCol("index") .setOutputCol("encoded") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala index f372ec58269e..ddb51fb1706a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg._ +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row class PCASuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { @@ -45,11 +46,11 @@ class PCASuite extends SparkFunSuite with MLlibTestSparkContext with DefaultRead val dataRDD = sc.parallelize(data, 2) - val mat = new RowMatrix(dataRDD) + val mat = new RowMatrix(dataRDD.map(OldVectors.fromML)) val pc = mat.computePrincipalComponents(3) - val expected = mat.multiply(pc).rows + val expected = mat.multiply(pc).rows.map(_.asML) - val df = sqlContext.createDataFrame(dataRDD.zip(expected)).toDF("features", "expected") + val df = spark.createDataFrame(dataRDD.zip(expected)).toDF("features", "expected") val pca = new PCA() .setInputCol("features") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala index 86dbee1cf4a5..8e1f9ddb36cb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala @@ -20,11 +20,11 @@ package org.apache.spark.ml.feature import org.scalatest.exceptions.TestFailedException import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row class PolynomialExpansionSuite @@ -59,7 +59,7 @@ class PolynomialExpansionSuite Vectors.sparse(19, Array.empty, Array.empty)) test("Polynomial expansion with default parameter") { - val df = sqlContext.createDataFrame(data.zip(twoDegreeExpansion)).toDF("features", "expected") + val df = spark.createDataFrame(data.zip(twoDegreeExpansion)).toDF("features", "expected") val polynomialExpansion = new PolynomialExpansion() .setInputCol("features") @@ -76,7 +76,7 @@ class PolynomialExpansionSuite } test("Polynomial expansion with setter") { - val df = sqlContext.createDataFrame(data.zip(threeDegreeExpansion)).toDF("features", "expected") + val df = spark.createDataFrame(data.zip(threeDegreeExpansion)).toDF("features", "expected") val polynomialExpansion = new PolynomialExpansion() .setInputCol("features") @@ -94,7 +94,7 @@ class PolynomialExpansionSuite } test("Polynomial expansion with degree 1 is identity on vectors") { - val df = sqlContext.createDataFrame(data.zip(data)).toDF("features", "expected") + val df = spark.createDataFrame(data.zip(data)).toDF("features", "expected") val polynomialExpansion = new PolynomialExpansion() .setInputCol("features") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala index f8476953d8d2..c12ab8fe9efe 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala @@ -19,11 +19,11 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.Row +import org.apache.spark.sql.types.DoubleType class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { test("params") { @@ -32,12 +32,12 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("transform numeric data") { val formula = new RFormula().setFormula("id ~ v1 + v2") - val original = sqlContext.createDataFrame( + val original = spark.createDataFrame( Seq((0, 1.0, 3.0), (2, 2.0, 5.0))).toDF("id", "v1", "v2") val model = formula.fit(original) val result = model.transform(original) val resultSchema = model.transformSchema(original.schema) - val expected = sqlContext.createDataFrame( + val expected = spark.createDataFrame( Seq( (0, 1.0, 3.0, Vectors.dense(1.0, 3.0), 0.0), (2, 2.0, 5.0, Vectors.dense(2.0, 5.0), 2.0)) @@ -50,7 +50,7 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("features column already exists") { val formula = new RFormula().setFormula("y ~ x").setFeaturesCol("x") - val original = sqlContext.createDataFrame(Seq((0, 1.0), (2, 2.0))).toDF("x", "y") + val original = spark.createDataFrame(Seq((0, 1.0), (2, 2.0))).toDF("x", "y") intercept[IllegalArgumentException] { formula.fit(original) } @@ -61,16 +61,16 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("label column already exists") { val formula = new RFormula().setFormula("y ~ x").setLabelCol("y") - val original = sqlContext.createDataFrame(Seq((0, 1.0), (2, 2.0))).toDF("x", "y") + val original = spark.createDataFrame(Seq((0, 1.0), (2, 2.0))).toDF("x", "y") val model = formula.fit(original) val resultSchema = model.transformSchema(original.schema) assert(resultSchema.length == 3) assert(resultSchema.toString == model.transform(original).schema.toString) } - test("label column already exists but is not double type") { + test("label column already exists but is not numeric type") { val formula = new RFormula().setFormula("y ~ x").setLabelCol("y") - val original = sqlContext.createDataFrame(Seq((0, 1), (2, 2))).toDF("x", "y") + val original = spark.createDataFrame(Seq((0, true), (2, false))).toDF("x", "y") val model = formula.fit(original) intercept[IllegalArgumentException] { model.transformSchema(original.schema) @@ -82,7 +82,7 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("allow missing label column for test datasets") { val formula = new RFormula().setFormula("y ~ x").setLabelCol("label") - val original = sqlContext.createDataFrame(Seq((0, 1.0), (2, 2.0))).toDF("x", "_not_y") + val original = spark.createDataFrame(Seq((0, 1.0), (2, 2.0))).toDF("x", "_not_y") val model = formula.fit(original) val resultSchema = model.transformSchema(original.schema) assert(resultSchema.length == 3) @@ -91,14 +91,14 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul } test("allow empty label") { - val original = sqlContext.createDataFrame( + val original = spark.createDataFrame( Seq((1, 2.0, 3.0), (4, 5.0, 6.0), (7, 8.0, 9.0)) ).toDF("id", "a", "b") val formula = new RFormula().setFormula("~ a + b") val model = formula.fit(original) val result = model.transform(original) val resultSchema = model.transformSchema(original.schema) - val expected = sqlContext.createDataFrame( + val expected = spark.createDataFrame( Seq( (1, 2.0, 3.0, Vectors.dense(2.0, 3.0)), (4, 5.0, 6.0, Vectors.dense(5.0, 6.0)), @@ -110,13 +110,13 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("encodes string terms") { val formula = new RFormula().setFormula("id ~ a + b") - val original = sqlContext.createDataFrame( + val original = spark.createDataFrame( Seq((1, "foo", 4), (2, "bar", 4), (3, "bar", 5), (4, "baz", 5)) ).toDF("id", "a", "b") val model = formula.fit(original) val result = model.transform(original) val resultSchema = model.transformSchema(original.schema) - val expected = sqlContext.createDataFrame( + val expected = spark.createDataFrame( Seq( (1, "foo", 4, Vectors.dense(0.0, 1.0, 4.0), 1.0), (2, "bar", 4, Vectors.dense(1.0, 0.0, 4.0), 2.0), @@ -129,13 +129,12 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("index string label") { val formula = new RFormula().setFormula("id ~ a + b") - val original = sqlContext.createDataFrame( + val original = spark.createDataFrame( Seq(("male", "foo", 4), ("female", "bar", 4), ("female", "bar", 5), ("male", "baz", 5)) ).toDF("id", "a", "b") val model = formula.fit(original) val result = model.transform(original) - val resultSchema = model.transformSchema(original.schema) - val expected = sqlContext.createDataFrame( + val expected = spark.createDataFrame( Seq( ("male", "foo", 4, Vectors.dense(0.0, 1.0, 4.0), 1.0), ("female", "bar", 4, Vectors.dense(1.0, 0.0, 4.0), 0.0), @@ -148,7 +147,7 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("attribute generation") { val formula = new RFormula().setFormula("id ~ a + b") - val original = sqlContext.createDataFrame( + val original = spark.createDataFrame( Seq((1, "foo", 4), (2, "bar", 4), (3, "bar", 5), (4, "baz", 5)) ).toDF("id", "a", "b") val model = formula.fit(original) @@ -165,7 +164,7 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("vector attribute generation") { val formula = new RFormula().setFormula("id ~ vec") - val original = sqlContext.createDataFrame( + val original = spark.createDataFrame( Seq((1, Vectors.dense(0.0, 1.0)), (2, Vectors.dense(1.0, 2.0))) ).toDF("id", "vec") val model = formula.fit(original) @@ -181,14 +180,14 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("vector attribute generation with unnamed input attrs") { val formula = new RFormula().setFormula("id ~ vec2") - val base = sqlContext.createDataFrame( + val base = spark.createDataFrame( Seq((1, Vectors.dense(0.0, 1.0)), (2, Vectors.dense(1.0, 2.0))) ).toDF("id", "vec") val metadata = new AttributeGroup( "vec2", Array[Attribute]( NumericAttribute.defaultAttr, - NumericAttribute.defaultAttr)).toMetadata + NumericAttribute.defaultAttr)).toMetadata() val original = base.select(base.col("id"), base.col("vec").as("vec2", metadata)) val model = formula.fit(original) val result = model.transform(original) @@ -203,12 +202,12 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("numeric interaction") { val formula = new RFormula().setFormula("a ~ b:c:d") - val original = sqlContext.createDataFrame( + val original = spark.createDataFrame( Seq((1, 2, 4, 2), (2, 3, 4, 1)) ).toDF("a", "b", "c", "d") val model = formula.fit(original) val result = model.transform(original) - val expected = sqlContext.createDataFrame( + val expected = spark.createDataFrame( Seq( (1, 2, 4, 2, Vectors.dense(16.0), 1.0), (2, 3, 4, 1, Vectors.dense(12.0), 2.0)) @@ -223,12 +222,12 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("factor numeric interaction") { val formula = new RFormula().setFormula("id ~ a:b") - val original = sqlContext.createDataFrame( + val original = spark.createDataFrame( Seq((1, "foo", 4), (2, "bar", 4), (3, "bar", 5), (4, "baz", 5), (4, "baz", 5), (4, "baz", 5)) ).toDF("id", "a", "b") val model = formula.fit(original) val result = model.transform(original) - val expected = sqlContext.createDataFrame( + val expected = spark.createDataFrame( Seq( (1, "foo", 4, Vectors.dense(0.0, 0.0, 4.0), 1.0), (2, "bar", 4, Vectors.dense(0.0, 4.0, 0.0), 2.0), @@ -250,12 +249,12 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("factor factor interaction") { val formula = new RFormula().setFormula("id ~ a:b") - val original = sqlContext.createDataFrame( + val original = spark.createDataFrame( Seq((1, "foo", "zq"), (2, "bar", "zq"), (3, "bar", "zz")) ).toDF("id", "a", "b") val model = formula.fit(original) val result = model.transform(original) - val expected = sqlContext.createDataFrame( + val expected = spark.createDataFrame( Seq( (1, "foo", "zq", Vectors.dense(0.0, 0.0, 1.0, 0.0), 1.0), (2, "bar", "zq", Vectors.dense(1.0, 0.0, 0.0, 0.0), 2.0), @@ -299,7 +298,7 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul } } - val dataset = sqlContext.createDataFrame( + val dataset = spark.createDataFrame( Seq((1, "foo", "zq"), (2, "bar", "zq"), (3, "bar", "zz")) ).toDF("id", "a", "b") @@ -309,4 +308,23 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul val newModel = testDefaultReadWrite(model) checkModelData(model, newModel) } + + test("should support all NumericType labels") { + val formula = new RFormula().setFormula("label ~ features") + .setLabelCol("x") + .setFeaturesCol("y") + val dfs = MLTestingUtils.genRegressionDFWithNumericLabelCol(spark) + val expected = formula.fit(dfs(DoubleType)) + val actuals = dfs.keys.filter(_ != DoubleType).map(t => formula.fit(dfs(t))) + actuals.foreach { actual => + assert(expected.pipelineModel.stages.length === actual.pipelineModel.stages.length) + expected.pipelineModel.stages.zip(actual.pipelineModel.stages).foreach { + case (exTransformer, acTransformer) => + assert(exTransformer.params === acTransformer.params) + } + assert(expected.resolvedFormula.label === actual.resolvedFormula.label) + assert(expected.resolvedFormula.terms === actual.resolvedFormula.terms) + assert(expected.resolvedFormula.hasIntercept === actual.resolvedFormula.hasIntercept) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala index e213e17d0d9d..1401ea9c4b43 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala @@ -31,13 +31,13 @@ class SQLTransformerSuite } test("transform numeric data") { - val original = sqlContext.createDataFrame( + val original = spark.createDataFrame( Seq((0, 1.0, 3.0), (2, 2.0, 5.0))).toDF("id", "v1", "v2") val sqlTrans = new SQLTransformer().setStatement( "SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__") val result = sqlTrans.transform(original) val resultSchema = sqlTrans.transformSchema(original.schema) - val expected = sqlContext.createDataFrame( + val expected = spark.createDataFrame( Seq((0, 1.0, 3.0, 4.0, 3.0), (2, 2.0, 5.0, 7.0, 10.0))) .toDF("id", "v1", "v2", "v3", "v4") assert(result.schema.toString == resultSchema.toString) @@ -52,7 +52,7 @@ class SQLTransformerSuite } test("transformSchema") { - val df = sqlContext.range(10) + val df = spark.range(10) val outputSchema = new SQLTransformer() .setStatement("SELECT id + 1 AS id1 FROM __THIS__") .transformSchema(df.schema) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StandardScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StandardScalerSuite.scala index 8c5e47a22c96..2243a0f972d3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/StandardScalerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StandardScalerSuite.scala @@ -18,11 +18,11 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} class StandardScalerSuite extends SparkFunSuite with MLlibTestSparkContext @@ -73,7 +73,7 @@ class StandardScalerSuite extends SparkFunSuite with MLlibTestSparkContext } test("Standardization with default parameter") { - val df0 = sqlContext.createDataFrame(data.zip(resWithStd)).toDF("features", "expected") + val df0 = spark.createDataFrame(data.zip(resWithStd)).toDF("features", "expected") val standardScaler0 = new StandardScaler() .setInputCol("features") @@ -84,9 +84,9 @@ class StandardScalerSuite extends SparkFunSuite with MLlibTestSparkContext } test("Standardization with setter") { - val df1 = sqlContext.createDataFrame(data.zip(resWithBoth)).toDF("features", "expected") - val df2 = sqlContext.createDataFrame(data.zip(resWithMean)).toDF("features", "expected") - val df3 = sqlContext.createDataFrame(data.zip(data)).toDF("features", "expected") + val df1 = spark.createDataFrame(data.zip(resWithBoth)).toDF("features", "expected") + val df2 = spark.createDataFrame(data.zip(resWithMean)).toDF("features", "expected") + val df3 = spark.createDataFrame(data.zip(data)).toDF("features", "expected") val standardScaler1 = new StandardScaler() .setInputCol("features") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala old mode 100644 new mode 100755 index 3505befdf8e3..125ad02ebcc0 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{Dataset, Row} object StopWordsRemoverSuite extends SparkFunSuite { def testStopWordsRemover(t: StopWordsRemover, dataset: Dataset[_]): Unit = { @@ -42,8 +42,26 @@ class StopWordsRemoverSuite val remover = new StopWordsRemover() .setInputCol("raw") .setOutputCol("filtered") - val dataSet = sqlContext.createDataFrame(Seq( + val dataSet = spark.createDataFrame(Seq( (Seq("test", "test"), Seq("test", "test")), + (Seq("a", "b", "c", "d"), Seq("b", "c")), + (Seq("a", "the", "an"), Seq()), + (Seq("A", "The", "AN"), Seq()), + (Seq(null), Seq(null)), + (Seq(), Seq()) + )).toDF("raw", "expected") + + testStopWordsRemover(remover, dataSet) + } + + test("StopWordsRemover with particular stop words list") { + val stopWords = Array("test", "a", "an", "the") + val remover = new StopWordsRemover() + .setInputCol("raw") + .setOutputCol("filtered") + .setStopWords(stopWords) + val dataSet = spark.createDataFrame(Seq( + (Seq("test", "test"), Seq()), (Seq("a", "b", "c", "d"), Seq("b", "c", "d")), (Seq("a", "the", "an"), Seq()), (Seq("A", "The", "AN"), Seq()), @@ -59,7 +77,7 @@ class StopWordsRemoverSuite .setInputCol("raw") .setOutputCol("filtered") .setCaseSensitive(true) - val dataSet = sqlContext.createDataFrame(Seq( + val dataSet = spark.createDataFrame(Seq( (Seq("A"), Seq("A")), (Seq("The", "the"), Seq("The")) )).toDF("raw", "expected") @@ -67,13 +85,48 @@ class StopWordsRemoverSuite testStopWordsRemover(remover, dataSet) } - test("StopWordsRemover with additional words") { - val stopWords = StopWords.English ++ Array("python", "scala") + test("default stop words of supported languages are not empty") { + StopWordsRemover.supportedLanguages.foreach { lang => + assert(StopWordsRemover.loadDefaultStopWords(lang).nonEmpty, + s"The default stop words of $lang cannot be empty.") + } + } + + test("StopWordsRemover with language selection") { + val stopWords = StopWordsRemover.loadDefaultStopWords("turkish") val remover = new StopWordsRemover() .setInputCol("raw") .setOutputCol("filtered") .setStopWords(stopWords) - val dataSet = sqlContext.createDataFrame(Seq( + val dataSet = spark.createDataFrame(Seq( + (Seq("acaba", "ama", "biri"), Seq()), + (Seq("hep", "her", "scala"), Seq("scala")) + )).toDF("raw", "expected") + + testStopWordsRemover(remover, dataSet) + } + + test("StopWordsRemover with ignored words") { + val stopWords = StopWordsRemover.loadDefaultStopWords("english").toSet -- Set("a") + val remover = new StopWordsRemover() + .setInputCol("raw") + .setOutputCol("filtered") + .setStopWords(stopWords.toArray) + val dataSet = spark.createDataFrame(Seq( + (Seq("python", "scala", "a"), Seq("python", "scala", "a")), + (Seq("Python", "Scala", "swift"), Seq("Python", "Scala", "swift")) + )).toDF("raw", "expected") + + testStopWordsRemover(remover, dataSet) + } + + test("StopWordsRemover with additional words") { + val stopWords = StopWordsRemover.loadDefaultStopWords("english").toSet ++ Set("python", "scala") + val remover = new StopWordsRemover() + .setInputCol("raw") + .setOutputCol("filtered") + .setStopWords(stopWords.toArray) + val dataSet = spark.createDataFrame(Seq( (Seq("python", "scala", "a"), Seq()), (Seq("Python", "Scala", "swift"), Seq("swift")) )).toDF("raw", "expected") @@ -95,7 +148,7 @@ class StopWordsRemoverSuite val remover = new StopWordsRemover() .setInputCol("raw") .setOutputCol(outputCol) - val dataSet = sqlContext.createDataFrame(Seq( + val dataSet = spark.createDataFrame(Seq( (Seq("The", "the", "swift"), Seq("swift")) )).toDF("raw", outputCol) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala index d0f3cdc841d1..c221d4aa558a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala @@ -39,7 +39,7 @@ class StringIndexerSuite test("StringIndexer") { val data = sc.parallelize(Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")), 2) - val df = sqlContext.createDataFrame(data).toDF("id", "label") + val df = spark.createDataFrame(data).toDF("id", "label") val indexer = new StringIndexer() .setInputCol("label") .setOutputCol("labelIndex") @@ -63,8 +63,8 @@ class StringIndexerSuite test("StringIndexerUnseen") { val data = sc.parallelize(Seq((0, "a"), (1, "b"), (4, "b")), 2) val data2 = sc.parallelize(Seq((0, "a"), (1, "b"), (2, "c")), 2) - val df = sqlContext.createDataFrame(data).toDF("id", "label") - val df2 = sqlContext.createDataFrame(data2).toDF("id", "label") + val df = spark.createDataFrame(data).toDF("id", "label") + val df2 = spark.createDataFrame(data2).toDF("id", "label") val indexer = new StringIndexer() .setInputCol("label") .setOutputCol("labelIndex") @@ -93,7 +93,7 @@ class StringIndexerSuite test("StringIndexer with a numeric input column") { val data = sc.parallelize(Seq((0, 100), (1, 200), (2, 300), (3, 100), (4, 100), (5, 300)), 2) - val df = sqlContext.createDataFrame(data).toDF("id", "label") + val df = spark.createDataFrame(data).toDF("id", "label") val indexer = new StringIndexer() .setInputCol("label") .setOutputCol("labelIndex") @@ -114,12 +114,12 @@ class StringIndexerSuite val indexerModel = new StringIndexerModel("indexer", Array("a", "b", "c")) .setInputCol("label") .setOutputCol("labelIndex") - val df = sqlContext.range(0L, 10L).toDF() + val df = spark.range(0L, 10L).toDF() assert(indexerModel.transform(df).collect().toSet === df.collect().toSet) } test("StringIndexerModel can't overwrite output column") { - val df = sqlContext.createDataFrame(Seq((1, 2), (3, 4))).toDF("input", "output") + val df = spark.createDataFrame(Seq((1, 2), (3, 4))).toDF("input", "output") val indexer = new StringIndexer() .setInputCol("input") .setOutputCol("output") @@ -153,7 +153,7 @@ class StringIndexerSuite test("IndexToString.transform") { val labels = Array("a", "b", "c") - val df0 = sqlContext.createDataFrame(Seq( + val df0 = spark.createDataFrame(Seq( (0, "a"), (1, "b"), (2, "c"), (0, "a") )).toDF("index", "expected") @@ -180,7 +180,7 @@ class StringIndexerSuite test("StringIndexer, IndexToString are inverses") { val data = sc.parallelize(Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")), 2) - val df = sqlContext.createDataFrame(data).toDF("id", "label") + val df = spark.createDataFrame(data).toDF("id", "label") val indexer = new StringIndexer() .setInputCol("label") .setOutputCol("labelIndex") @@ -213,7 +213,7 @@ class StringIndexerSuite test("StringIndexer metadata") { val data = sc.parallelize(Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")), 2) - val df = sqlContext.createDataFrame(data).toDF("id", "label") + val df = spark.createDataFrame(data).toDF("id", "label") val indexer = new StringIndexer() .setInputCol("label") .setOutputCol("labelIndex") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala index 123ddfe42c95..f30bdc3ddc0d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala @@ -57,13 +57,13 @@ class RegexTokenizerSuite .setPattern("\\w+|\\p{Punct}") .setInputCol("rawText") .setOutputCol("tokens") - val dataset0 = sqlContext.createDataFrame(Seq( + val dataset0 = spark.createDataFrame(Seq( TokenizerTestData("Test for tokenization.", Array("test", "for", "tokenization", ".")), TokenizerTestData("Te,st. punct", Array("te", ",", "st", ".", "punct")) )) testRegexTokenizer(tokenizer0, dataset0) - val dataset1 = sqlContext.createDataFrame(Seq( + val dataset1 = spark.createDataFrame(Seq( TokenizerTestData("Test for tokenization.", Array("test", "for", "tokenization")), TokenizerTestData("Te,st. punct", Array("punct")) )) @@ -73,7 +73,7 @@ class RegexTokenizerSuite val tokenizer2 = new RegexTokenizer() .setInputCol("rawText") .setOutputCol("tokens") - val dataset2 = sqlContext.createDataFrame(Seq( + val dataset2 = spark.createDataFrame(Seq( TokenizerTestData("Test for tokenization.", Array("test", "for", "tokenization.")), TokenizerTestData("Te,st. punct", Array("te,st.", "punct")) )) @@ -85,7 +85,7 @@ class RegexTokenizerSuite .setInputCol("rawText") .setOutputCol("tokens") .setToLowercase(false) - val dataset = sqlContext.createDataFrame(Seq( + val dataset = spark.createDataFrame(Seq( TokenizerTestData("JAVA SCALA", Array("JAVA", "SCALA")), TokenizerTestData("java scala", Array("java", "scala")) )) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala index dce994fdbd05..14973e79bf34 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.feature import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.attribute.{AttributeGroup, NominalAttribute, NumericAttribute} +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.Row import org.apache.spark.sql.functions.col @@ -57,7 +57,7 @@ class VectorAssemblerSuite } test("VectorAssembler") { - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, 0.0, Vectors.dense(1.0, 2.0), "a", Vectors.sparse(2, Array(1), Array(3.0)), 10L) )).toDF("id", "x", "y", "name", "z", "n") val assembler = new VectorAssembler() @@ -70,7 +70,7 @@ class VectorAssemblerSuite } test("transform should throw an exception in case of unsupported type") { - val df = sqlContext.createDataFrame(Seq(("a", "b", "c"))).toDF("a", "b", "c") + val df = spark.createDataFrame(Seq(("a", "b", "c"))).toDF("a", "b", "c") val assembler = new VectorAssembler() .setInputCols(Array("a", "b", "c")) .setOutputCol("features") @@ -87,7 +87,7 @@ class VectorAssemblerSuite NominalAttribute.defaultAttr.withName("gender").withValues("male", "female"), NumericAttribute.defaultAttr.withName("salary"))) val row = (1.0, 0.5, 1, Vectors.dense(1.0, 1000.0), Vectors.sparse(2, Array(1), Array(2.0))) - val df = sqlContext.createDataFrame(Seq(row)).toDF("browser", "hour", "count", "user", "ad") + val df = spark.createDataFrame(Seq(row)).toDF("browser", "hour", "count", "user", "ad") .select( col("browser").as("browser", browser.toMetadata()), col("hour").as("hour", hour.toMetadata()), diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala index 1ffc62b38e85..707142332349 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala @@ -22,9 +22,9 @@ import scala.beans.{BeanInfo, BeanProperty} import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.internal.Logging import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.{SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame @@ -85,11 +85,11 @@ class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext checkPair(densePoints1Seq, sparsePoints1Seq) checkPair(densePoints2Seq, sparsePoints2Seq) - densePoints1 = sqlContext.createDataFrame(sc.parallelize(densePoints1Seq, 2).map(FeatureData)) - sparsePoints1 = sqlContext.createDataFrame(sc.parallelize(sparsePoints1Seq, 2).map(FeatureData)) - densePoints2 = sqlContext.createDataFrame(sc.parallelize(densePoints2Seq, 2).map(FeatureData)) - sparsePoints2 = sqlContext.createDataFrame(sc.parallelize(sparsePoints2Seq, 2).map(FeatureData)) - badPoints = sqlContext.createDataFrame(sc.parallelize(badPointsSeq, 2).map(FeatureData)) + densePoints1 = spark.createDataFrame(sc.parallelize(densePoints1Seq, 2).map(FeatureData)) + sparsePoints1 = spark.createDataFrame(sc.parallelize(sparsePoints1Seq, 2).map(FeatureData)) + densePoints2 = spark.createDataFrame(sc.parallelize(densePoints2Seq, 2).map(FeatureData)) + sparsePoints2 = spark.createDataFrame(sc.parallelize(sparsePoints2Seq, 2).map(FeatureData)) + badPoints = spark.createDataFrame(sc.parallelize(badPointsSeq, 2).map(FeatureData)) } private def getIndexer: VectorIndexer = @@ -102,7 +102,7 @@ class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext } test("Cannot fit an empty DataFrame") { - val rdd = sqlContext.createDataFrame(sc.parallelize(Array.empty[Vector], 2).map(FeatureData)) + val rdd = spark.createDataFrame(sc.parallelize(Array.empty[Vector], 2).map(FeatureData)) val vectorIndexer = getIndexer intercept[IllegalArgumentException] { vectorIndexer.fit(rdd) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala index 6bb4678dc5f9..1746ce53107c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.types.{StructField, StructType} @@ -79,7 +79,7 @@ class VectorSlicerSuite extends SparkFunSuite with MLlibTestSparkContext with De val resultAttrGroup = new AttributeGroup("expected", resultAttrs.asInstanceOf[Array[Attribute]]) val rdd = sc.parallelize(data.zip(expected)).map { case (a, b) => Row(a, b) } - val df = sqlContext.createDataFrame(rdd, + val df = spark.createDataFrame(rdd, StructType(Array(attrGroup.toStructField(), resultAttrGroup.toStructField()))) val vectorSlicer = new VectorSlicer().setInputCol("features").setOutputCol("result") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala index 80c177b8d318..280a36f56e93 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala @@ -18,12 +18,12 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.feature.{Word2VecModel => OldWord2VecModel} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { @@ -36,8 +36,8 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("Word2Vec") { - val sqlContext = this.sqlContext - import sqlContext.implicits._ + val spark = this.spark + import spark.implicits._ val sentence = "a b " * 100 + "a c " * 10 val numOfWords = sentence.split(" ").size @@ -78,8 +78,8 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("getVectors") { - val sqlContext = this.sqlContext - import sqlContext.implicits._ + val spark = this.spark + import spark.implicits._ val sentence = "a b " * 100 + "a c " * 10 val doc = sc.parallelize(Seq(sentence, sentence)).map(line => line.split(" ")) @@ -119,8 +119,8 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("findSynonyms") { - val sqlContext = this.sqlContext - import sqlContext.implicits._ + val spark = this.spark + import spark.implicits._ val sentence = "a b " * 100 + "a c " * 10 val doc = sc.parallelize(Seq(sentence, sentence)).map(line => line.split(" ")) @@ -146,8 +146,8 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul test("window size") { - val sqlContext = this.sqlContext - import sqlContext.implicits._ + val spark = this.spark + import spark.implicits._ val sentence = "a q s t q s t b b b s t m s t m q " * 100 + "a c " * 10 val doc = sc.parallelize(Seq(sentence, sentence)).map(line => line.split(" ")) diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala index 6d01d8f2828e..6ddb12cb76aa 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.ml.linalg import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.sql.catalyst.JavaTypeInference import org.apache.spark.sql.types._ class VectorUDTSuite extends SparkFunSuite { @@ -36,4 +38,10 @@ class VectorUDTSuite extends SparkFunSuite { assert(udt.simpleString == "vector") } } + + test("JavaTypeInference with VectorUDT") { + val (dataType, _) = JavaTypeInference.inferDataType(classOf[LabeledPoint]) + assert(dataType.asInstanceOf[StructType].fields.map(_.dataType) + === Seq(new VectorUDT, DoubleType)) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala index 604021220a13..b30d995794d4 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.optim import org.apache.spark.SparkFunSuite import org.apache.spark.ml.feature.Instance -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD class IterativelyReweightedLeastSquaresSuite extends SparkFunSuite with MLlibTestSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala index 0b58a9821f57..c8de796b2de8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.optim import org.apache.spark.SparkFunSuite import org.apache.spark.ml.feature.Instance -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD class WeightedLeastSquaresSuite extends SparkFunSuite with MLlibTestSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala index a3366c0e5934..aa9c53ca30ee 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.ml.param -import java.io.{ByteArrayOutputStream, NotSerializableException, ObjectOutputStream} +import java.io.{ByteArrayOutputStream, ObjectOutputStream} import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.MyParams -import org.apache.spark.mllib.linalg.{Vector, Vectors} class ParamsSuite extends SparkFunSuite { diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index 4c4eb72cd16e..59b5edc4013e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -17,25 +17,31 @@ package org.apache.spark.ml.recommendation +import java.io.File import java.util.Random import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConverters._ import scala.language.existentials import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.apache.commons.io.FileUtils +import org.apache.commons.io.filefilter.TrueFileFilter -import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.recommendation.ALS._ import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted} -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.types.{FloatType, IntegerType} import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class ALSSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest with Logging { @@ -200,7 +206,6 @@ class ALSSuite /** * Generates an explicit feedback dataset for testing ALS. - * * @param numUsers number of users * @param numItems number of items * @param rank rank @@ -241,7 +246,6 @@ class ALSSuite /** * Generates an implicit feedback dataset for testing ALS. - * * @param numUsers number of users * @param numItems number of items * @param rank rank @@ -255,42 +259,11 @@ class ALSSuite rank: Int, noiseStd: Double = 0.0, seed: Long = 11L): (RDD[Rating[Int]], RDD[Rating[Int]]) = { - // The assumption of the implicit feedback model is that unobserved ratings are more likely to - // be negatives. - val positiveFraction = 0.8 - val negativeFraction = 1.0 - positiveFraction - val trainingFraction = 0.6 - val testFraction = 0.3 - val totalFraction = trainingFraction + testFraction - val random = new Random(seed) - val userFactors = genFactors(numUsers, rank, random) - val itemFactors = genFactors(numItems, rank, random) - val training = ArrayBuffer.empty[Rating[Int]] - val test = ArrayBuffer.empty[Rating[Int]] - for ((userId, userFactor) <- userFactors; (itemId, itemFactor) <- itemFactors) { - val rating = blas.sdot(rank, userFactor, 1, itemFactor, 1) - val threshold = if (rating > 0) positiveFraction else negativeFraction - val observed = random.nextDouble() < threshold - if (observed) { - val x = random.nextDouble() - if (x < totalFraction) { - if (x < trainingFraction) { - val noise = noiseStd * random.nextGaussian() - training += Rating(userId, itemId, rating + noise.toFloat) - } else { - test += Rating(userId, itemId, rating) - } - } - } - } - logInfo(s"Generated an implicit feedback dataset with ${training.size} ratings for training " + - s"and ${test.size} for test.") - (sc.parallelize(training, 2), sc.parallelize(test, 2)) + ALSSuite.genImplicitTestData(sc, numUsers, numItems, rank, noiseStd, seed) } /** * Generates random user/item factors, with i.i.d. values drawn from U(a, b). - * * @param size number of users/items * @param rank number of features * @param random random number generator @@ -304,19 +277,11 @@ class ALSSuite random: Random, a: Float = -1.0f, b: Float = 1.0f): Seq[(Int, Array[Float])] = { - require(size > 0 && size < Int.MaxValue / 3) - require(b > a) - val ids = mutable.Set.empty[Int] - while (ids.size < size) { - ids += random.nextInt() - } - val width = b - a - ids.toSeq.sorted.map(id => (id, Array.fill(rank)(a + random.nextFloat() * width))) + ALSSuite.genFactors(size, rank, random, a, b) } /** * Test ALS using the given training/test splits and parameters. - * * @param training training dataset * @param test test dataset * @param rank rank of the matrix factorization @@ -337,8 +302,8 @@ class ALSSuite numUserBlocks: Int = 2, numItemBlocks: Int = 3, targetRMSE: Double = 0.05): Unit = { - val sqlContext = this.sqlContext - import sqlContext.implicits._ + val spark = this.spark + import spark.implicits._ val als = new ALS() .setRank(rank) .setRegParam(regParam) @@ -492,8 +457,8 @@ class ALSSuite allEstimatorParamSettings.foreach { case (p, v) => als.set(als.getParam(p), v) } - val sqlContext = this.sqlContext - import sqlContext.implicits._ + val spark = this.spark + import spark.implicits._ val model = als.fit(ratings.toDF()) // Test Estimator save/load @@ -518,6 +483,138 @@ class ALSSuite assert(getFactors(model.userFactors) === getFactors(model2.userFactors)) assert(getFactors(model.itemFactors) === getFactors(model2.itemFactors)) } + + test("input type validation") { + val spark = this.spark + import spark.implicits._ + + // check that ALS can handle all numeric types for rating column + // and user/item columns (when the user/item ids are within Int range) + val als = new ALS().setMaxIter(1).setRank(1) + Seq(("user", IntegerType), ("item", IntegerType), ("rating", FloatType)).foreach { + case (colName, sqlType) => + MLTestingUtils.checkNumericTypesALS(als, spark, colName, sqlType) { + (ex, act) => + ex.userFactors.first().getSeq[Float](1) === act.userFactors.first.getSeq[Float](1) + } { (ex, act, _) => + ex.transform(_: DataFrame).select("prediction").first.getFloat(0) ~== + act.transform(_: DataFrame).select("prediction").first.getFloat(0) absTol 1e-6 + } + } + // check user/item ids falling outside of Int range + val big = Int.MaxValue.toLong + 1 + val small = Int.MinValue.toDouble - 1 + val df = Seq( + (0, 0L, 0d, 1, 1L, 1d, 3.0), + (0, big, small, 0, big, small, 2.0), + (1, 1L, 1d, 0, 0L, 0d, 5.0) + ).toDF("user", "user_big", "user_small", "item", "item_big", "item_small", "rating") + withClue("fit should fail when ids exceed integer range. ") { + assert(intercept[IllegalArgumentException] { + als.fit(df.select(df("user_big").as("user"), df("item"), df("rating"))) + }.getMessage.contains("was out of Integer range")) + assert(intercept[IllegalArgumentException] { + als.fit(df.select(df("user_small").as("user"), df("item"), df("rating"))) + }.getMessage.contains("was out of Integer range")) + assert(intercept[IllegalArgumentException] { + als.fit(df.select(df("item_big").as("item"), df("user"), df("rating"))) + }.getMessage.contains("was out of Integer range")) + assert(intercept[IllegalArgumentException] { + als.fit(df.select(df("item_small").as("item"), df("user"), df("rating"))) + }.getMessage.contains("was out of Integer range")) + } + withClue("transform should fail when ids exceed integer range. ") { + val model = als.fit(df) + assert(intercept[SparkException] { + model.transform(df.select(df("user_big").as("user"), df("item"))).first + }.getMessage.contains("was out of Integer range")) + assert(intercept[SparkException] { + model.transform(df.select(df("user_small").as("user"), df("item"))).first + }.getMessage.contains("was out of Integer range")) + assert(intercept[SparkException] { + model.transform(df.select(df("item_big").as("item"), df("user"))).first + }.getMessage.contains("was out of Integer range")) + assert(intercept[SparkException] { + model.transform(df.select(df("item_small").as("item"), df("user"))).first + }.getMessage.contains("was out of Integer range")) + } + } +} + +class ALSCleanerSuite extends SparkFunSuite { + test("ALS shuffle cleanup standalone") { + val conf = new SparkConf() + val localDir = Utils.createTempDir() + val checkpointDir = Utils.createTempDir() + def getAllFiles: Set[File] = + FileUtils.listFiles(localDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).asScala.toSet + try { + conf.set("spark.local.dir", localDir.getAbsolutePath) + val sc = new SparkContext("local[2]", "test", conf) + try { + sc.setCheckpointDir(checkpointDir.getAbsolutePath) + // Test checkpoint and clean parents + val input = sc.parallelize(1 to 1000) + val keyed = input.map(x => (x % 20, 1)) + val shuffled = keyed.reduceByKey(_ + _) + val keysOnly = shuffled.keys + val deps = keysOnly.dependencies + keysOnly.count() + ALS.cleanShuffleDependencies(sc, deps, true) + val resultingFiles = getAllFiles + assert(resultingFiles === Set()) + // Ensure running count again works fine even if we kill the shuffle files. + keysOnly.count() + } finally { + sc.stop() + } + } finally { + Utils.deleteRecursively(localDir) + Utils.deleteRecursively(checkpointDir) + } + } + + test("ALS shuffle cleanup in algorithm") { + val conf = new SparkConf() + val localDir = Utils.createTempDir() + val checkpointDir = Utils.createTempDir() + def getAllFiles: Set[File] = + FileUtils.listFiles(localDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).asScala.toSet + try { + conf.set("spark.local.dir", localDir.getAbsolutePath) + val sc = new SparkContext("local[2]", "test", conf) + try { + sc.setCheckpointDir(checkpointDir.getAbsolutePath) + // Generate test data + val (training, _) = ALSSuite.genImplicitTestData(sc, 20, 5, 1, 0.2, 0) + // Implicitly test the cleaning of parents during ALS training + val spark = SparkSession.builder + .master("local[2]") + .appName("ALSCleanerSuite") + .getOrCreate() + import spark.implicits._ + val als = new ALS() + .setRank(1) + .setRegParam(1e-5) + .setSeed(0) + .setCheckpointInterval(1) + .setMaxIter(7) + val model = als.fit(training.toDF()) + val resultingFiles = getAllFiles + // We expect the last shuffles files, block ratings, user factors, and item factors to be + // around but no more. + val pattern = "shuffle_(\\d+)_.+\\.data".r + val rddIds = resultingFiles.flatMap { f => + pattern.findAllIn(f.getName()).matchData.map { _.group(1) } } + assert(rddIds.toSet.size === 4) + } finally { + sc.stop() + } + } finally { + Utils.deleteRecursively(localDir) + Utils.deleteRecursively(checkpointDir) + } + } } class ALSStorageSuite @@ -536,8 +633,8 @@ class ALSStorageSuite } test("default and non-default storage params set correct RDD StorageLevels") { - val sqlContext = this.sqlContext - import sqlContext.implicits._ + val spark = this.spark + import spark.implicits._ val data = Seq( (0, 0, 1.0), (0, 1, 2.0), @@ -591,7 +688,7 @@ private class IntermediateRDDStorageListener extends SparkListener { } -object ALSSuite { +object ALSSuite extends Logging { /** * Mapping from all Params to valid settings which differ from the defaults. @@ -620,4 +717,82 @@ object ALSSuite { "intermediateStorageLevel" -> "MEMORY_ONLY", "finalStorageLevel" -> "MEMORY_AND_DISK_SER" ) + + // Helper functions to generate test data we share between ALS test suites + + /** + * Generates random user/item factors, with i.i.d. values drawn from U(a, b). + * @param size number of users/items + * @param rank number of features + * @param random random number generator + * @param a min value of the support (default: -1) + * @param b max value of the support (default: 1) + * @return a sequence of (ID, factors) pairs + */ + private def genFactors( + size: Int, + rank: Int, + random: Random, + a: Float = -1.0f, + b: Float = 1.0f): Seq[(Int, Array[Float])] = { + require(size > 0 && size < Int.MaxValue / 3) + require(b > a) + val ids = mutable.Set.empty[Int] + while (ids.size < size) { + ids += random.nextInt() + } + val width = b - a + ids.toSeq.sorted.map(id => (id, Array.fill(rank)(a + random.nextFloat() * width))) + } + + /** + * Generates an implicit feedback dataset for testing ALS. + * + * @param sc SparkContext + * @param numUsers number of users + * @param numItems number of items + * @param rank rank + * @param noiseStd the standard deviation of additive Gaussian noise on training data + * @param seed random seed + * @return (training, test) + */ + def genImplicitTestData( + sc: SparkContext, + numUsers: Int, + numItems: Int, + rank: Int, + noiseStd: Double = 0.0, + seed: Long = 11L): (RDD[Rating[Int]], RDD[Rating[Int]]) = { + // The assumption of the implicit feedback model is that unobserved ratings are more likely to + // be negatives. + val positiveFraction = 0.8 + val negativeFraction = 1.0 - positiveFraction + val trainingFraction = 0.6 + val testFraction = 0.3 + val totalFraction = trainingFraction + testFraction + val random = new Random(seed) + val userFactors = genFactors(numUsers, rank, random) + val itemFactors = genFactors(numItems, rank, random) + val training = ArrayBuffer.empty[Rating[Int]] + val test = ArrayBuffer.empty[Rating[Int]] + for ((userId, userFactor) <- userFactors; (itemId, itemFactor) <- itemFactors) { + val rating = blas.sdot(rank, userFactor, 1, itemFactor, 1) + val threshold = if (rating > 0) positiveFraction else negativeFraction + val observed = random.nextDouble() < threshold + if (observed) { + val x = random.nextDouble() + if (x < totalFraction) { + if (x < trainingFraction) { + val noise = noiseStd * random.nextGaussian() + training += Rating(userId, itemId, rating + noise.toFloat) + } else { + test += Rating(userId, itemId, rating) + } + } + } + } + logInfo(s"Generated an implicit feedback dataset with ${training.size} ratings for training " + + s"and ${test.size} for test.") + (sc.parallelize(training, 2), sc.parallelize(test, 2)) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala index 76891ad56281..05aae80c660e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala @@ -20,12 +20,12 @@ package org.apache.spark.ml.regression import scala.util.Random import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.random.{ExponentialGenerator, WeibullGenerator} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} class AFTSurvivalRegressionSuite @@ -37,13 +37,13 @@ class AFTSurvivalRegressionSuite override def beforeAll(): Unit = { super.beforeAll() - datasetUnivariate = sqlContext.createDataFrame( + datasetUnivariate = spark.createDataFrame( sc.parallelize(generateAFTInput( 1, Array(5.5), Array(0.8), 1000, 42, 1.0, 2.0, 2.0))) - datasetMultivariate = sqlContext.createDataFrame( + datasetMultivariate = spark.createDataFrame( sc.parallelize(generateAFTInput( 2, Array(0.9, -1.3), Array(0.7, 1.2), 1000, 42, 1.5, 2.5, 2.0))) - datasetUnivariateScaled = sqlContext.createDataFrame( + datasetUnivariateScaled = spark.createDataFrame( sc.parallelize(generateAFTInput( 1, Array(5.5), Array(0.8), 1000, 42, 1.0, 2.0, 2.0)).map { x => AFTPoint(Vectors.dense(x.features(0) * 1.0E3), x.label, x.censor) @@ -356,7 +356,7 @@ class AFTSurvivalRegressionSuite test("should support all NumericType labels") { val aft = new AFTSurvivalRegression().setMaxIter(1) MLTestingUtils.checkNumericTypes[AFTSurvivalRegressionModel, AFTSurvivalRegression]( - aft, isClassification = false, sqlContext) { (expected, actual) => + aft, spark, isClassification = false) { (expected, actual) => assert(expected.intercept === actual.intercept) assert(expected.coefficients === actual.coefficients) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala index e9fb2677b215..9afb742406ec 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala @@ -18,10 +18,11 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree, DecisionTreeSuite => OldDecisionTreeSuite} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -38,7 +39,7 @@ class DecisionTreeRegressorSuite override def beforeAll() { super.beforeAll() categoricalDataPointsRDD = - sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()) + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints().map(_.asML)) } ///////////////////////////////////////////////////////////////////////////// @@ -120,7 +121,7 @@ class DecisionTreeRegressorSuite test("should support all NumericType labels and not support other types") { val dt = new DecisionTreeRegressor().setMaxDepth(1) MLTestingUtils.checkNumericTypes[DecisionTreeRegressionModel, DecisionTreeRegressor]( - dt, isClassification = false, sqlContext) { (expected, actual) => + dt, spark, isClassification = false) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } @@ -170,7 +171,7 @@ private[ml] object DecisionTreeRegressorSuite extends SparkFunSuite { categoricalFeatures: Map[Int, Int]): Unit = { val numFeatures = data.first().features.size val oldStrategy = dt.getOldStrategy(categoricalFeatures) - val oldTree = OldDecisionTree.train(data, oldStrategy) + val oldTree = OldDecisionTree.train(data.map(OldLabeledPoint.fromML), oldStrategy) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) val newTree = dt.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala index 216377959e09..7b5df8f31bb3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala @@ -18,10 +18,11 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, GradientBoostedTrees => OldGBT} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -48,10 +49,13 @@ class GBTRegressorSuite extends SparkFunSuite with MLlibTestSparkContext override def beforeAll() { super.beforeAll() data = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100), 2) + .map(_.asML) trainData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 120), 2) + .map(_.asML) validationData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 80), 2) + .map(_.asML) } test("Regression with continuous features") { @@ -72,7 +76,7 @@ class GBTRegressorSuite extends SparkFunSuite with MLlibTestSparkContext } test("GBTRegressor behaves reasonably on toy data") { - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( LabeledPoint(10, Vectors.dense(1, 2, 3, 4)), LabeledPoint(-5, Vectors.dense(6, 3, 2, 1)), LabeledPoint(11, Vectors.dense(2, 2, 3, 4)), @@ -99,7 +103,7 @@ class GBTRegressorSuite extends SparkFunSuite with MLlibTestSparkContext val path = tempDir.toURI.toString sc.setCheckpointDir(path) - val df = sqlContext.createDataFrame(data) + val df = spark.createDataFrame(data) val gbt = new GBTRegressor() .setMaxDepth(2) .setMaxIter(5) @@ -115,7 +119,7 @@ class GBTRegressorSuite extends SparkFunSuite with MLlibTestSparkContext test("should support all NumericType labels and not support other types") { val gbt = new GBTRegressor().setMaxDepth(1) MLTestingUtils.checkNumericTypes[GBTRegressionModel, GBTRegressor]( - gbt, isClassification = false, sqlContext) { (expected, actual) => + gbt, spark, isClassification = false) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } @@ -197,7 +201,7 @@ private object GBTRegressorSuite extends SparkFunSuite { val numFeatures = data.first().features.size val oldBoostingStrategy = gbt.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Regression) val oldGBT = new OldGBT(oldBoostingStrategy, gbt.getSeed.toInt) - val oldModel = oldGBT.run(data) + val oldModel = oldGBT.run(data.map(OldLabeledPoint.fromML)) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) val newModel = gbt.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala index b854be2f1fbc..a4568e83faca 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala @@ -20,15 +20,15 @@ package org.apache.spark.ml.regression import scala.util.Random import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.classification.LogisticRegressionSuite._ import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.classification.LogisticRegressionSuite._ -import org.apache.spark.mllib.linalg.{BLAS, DenseVector, Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.random._ -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.functions._ @@ -52,19 +52,19 @@ class GeneralizedLinearRegressionSuite import GeneralizedLinearRegressionSuite._ - datasetGaussianIdentity = sqlContext.createDataFrame( + datasetGaussianIdentity = spark.createDataFrame( sc.parallelize(generateGeneralizedLinearRegressionInput( intercept = 2.5, coefficients = Array(2.2, 0.6), xMean = Array(2.9, 10.5), xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, family = "gaussian", link = "identity"), 2)) - datasetGaussianLog = sqlContext.createDataFrame( + datasetGaussianLog = spark.createDataFrame( sc.parallelize(generateGeneralizedLinearRegressionInput( intercept = 0.25, coefficients = Array(0.22, 0.06), xMean = Array(2.9, 10.5), xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, family = "gaussian", link = "log"), 2)) - datasetGaussianInverse = sqlContext.createDataFrame( + datasetGaussianInverse = spark.createDataFrame( sc.parallelize(generateGeneralizedLinearRegressionInput( intercept = 2.5, coefficients = Array(2.2, 0.6), xMean = Array(2.9, 10.5), xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, @@ -80,40 +80,40 @@ class GeneralizedLinearRegressionSuite generateMultinomialLogisticInput(coefficients, xMean, xVariance, addIntercept = true, nPoints, seed) - sqlContext.createDataFrame(sc.parallelize(testData, 2)) + spark.createDataFrame(sc.parallelize(testData, 2)) } - datasetPoissonLog = sqlContext.createDataFrame( + datasetPoissonLog = spark.createDataFrame( sc.parallelize(generateGeneralizedLinearRegressionInput( intercept = 0.25, coefficients = Array(0.22, 0.06), xMean = Array(2.9, 10.5), xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, family = "poisson", link = "log"), 2)) - datasetPoissonIdentity = sqlContext.createDataFrame( + datasetPoissonIdentity = spark.createDataFrame( sc.parallelize(generateGeneralizedLinearRegressionInput( intercept = 2.5, coefficients = Array(2.2, 0.6), xMean = Array(2.9, 10.5), xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, family = "poisson", link = "identity"), 2)) - datasetPoissonSqrt = sqlContext.createDataFrame( + datasetPoissonSqrt = spark.createDataFrame( sc.parallelize(generateGeneralizedLinearRegressionInput( intercept = 2.5, coefficients = Array(2.2, 0.6), xMean = Array(2.9, 10.5), xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, family = "poisson", link = "sqrt"), 2)) - datasetGammaInverse = sqlContext.createDataFrame( + datasetGammaInverse = spark.createDataFrame( sc.parallelize(generateGeneralizedLinearRegressionInput( intercept = 2.5, coefficients = Array(2.2, 0.6), xMean = Array(2.9, 10.5), xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, family = "gamma", link = "inverse"), 2)) - datasetGammaIdentity = sqlContext.createDataFrame( + datasetGammaIdentity = spark.createDataFrame( sc.parallelize(generateGeneralizedLinearRegressionInput( intercept = 2.5, coefficients = Array(2.2, 0.6), xMean = Array(2.9, 10.5), xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, family = "gamma", link = "identity"), 2)) - datasetGammaLog = sqlContext.createDataFrame( + datasetGammaLog = spark.createDataFrame( sc.parallelize(generateGeneralizedLinearRegressionInput( intercept = 0.25, coefficients = Array(0.22, 0.06), xMean = Array(2.9, 10.5), xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, @@ -540,7 +540,7 @@ class GeneralizedLinearRegressionSuite w <- c(1, 2, 3, 4) df <- as.data.frame(cbind(A, b)) */ - val datasetWithWeight = sqlContext.createDataFrame(sc.parallelize(Seq( + val datasetWithWeight = spark.createDataFrame(sc.parallelize(Seq( Instance(17.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), Instance(19.0, 2.0, Vectors.dense(1.0, 7.0)), Instance(23.0, 3.0, Vectors.dense(2.0, 11.0)), @@ -668,7 +668,7 @@ class GeneralizedLinearRegressionSuite w <- c(1, 2, 3, 4) df <- as.data.frame(cbind(A, b)) */ - val datasetWithWeight = sqlContext.createDataFrame(sc.parallelize(Seq( + val datasetWithWeight = spark.createDataFrame(sc.parallelize(Seq( Instance(1.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), Instance(0.0, 2.0, Vectors.dense(1.0, 2.0)), Instance(1.0, 3.0, Vectors.dense(2.0, 1.0)), @@ -782,7 +782,7 @@ class GeneralizedLinearRegressionSuite w <- c(1, 2, 3, 4) df <- as.data.frame(cbind(A, b)) */ - val datasetWithWeight = sqlContext.createDataFrame(sc.parallelize(Seq( + val datasetWithWeight = spark.createDataFrame(sc.parallelize(Seq( Instance(2.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), Instance(8.0, 2.0, Vectors.dense(1.0, 7.0)), Instance(3.0, 3.0, Vectors.dense(2.0, 11.0)), @@ -899,7 +899,7 @@ class GeneralizedLinearRegressionSuite w <- c(1, 2, 3, 4) df <- as.data.frame(cbind(A, b)) */ - val datasetWithWeight = sqlContext.createDataFrame(sc.parallelize(Seq( + val datasetWithWeight = spark.createDataFrame(sc.parallelize(Seq( Instance(2.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), Instance(8.0, 2.0, Vectors.dense(1.0, 7.0)), Instance(3.0, 3.0, Vectors.dense(2.0, 11.0)), @@ -1021,14 +1021,14 @@ class GeneralizedLinearRegressionSuite val glr = new GeneralizedLinearRegression().setMaxIter(1) MLTestingUtils.checkNumericTypes[ GeneralizedLinearRegressionModel, GeneralizedLinearRegression]( - glr, isClassification = false, sqlContext) { (expected, actual) => + glr, spark, isClassification = false) { (expected, actual) => assert(expected.intercept === actual.intercept) assert(expected.coefficients === actual.coefficients) } } test("glm accepts Dataset[LabeledPoint]") { - val context = sqlContext + val context = spark import context.implicits._ new GeneralizedLinearRegression() .setFamily("gaussian") diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala index 3a10ad7ed060..14d8a4e4e334 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Row} @@ -28,13 +28,13 @@ class IsotonicRegressionSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { private def generateIsotonicInput(labels: Seq[Double]): DataFrame = { - sqlContext.createDataFrame( + spark.createDataFrame( labels.zipWithIndex.map { case (label, i) => (label, i.toDouble, 1.0) } ).toDF("label", "features", "weight") } private def generatePredictionInput(features: Seq[Double]): DataFrame = { - sqlContext.createDataFrame(features.map(Tuple1.apply)) + spark.createDataFrame(features.map(Tuple1.apply)) .toDF("features") } @@ -145,7 +145,7 @@ class IsotonicRegressionSuite } test("vector features column with feature index") { - val dataset = sqlContext.createDataFrame(Seq( + val dataset = spark.createDataFrame(Seq( (4.0, Vectors.dense(0.0, 1.0)), (3.0, Vectors.dense(0.0, 2.0)), (5.0, Vectors.sparse(2, Array(1), Array(3.0)))) @@ -184,7 +184,7 @@ class IsotonicRegressionSuite test("should support all NumericType labels and not support other types") { val ir = new IsotonicRegression() MLTestingUtils.checkNumericTypes[IsotonicRegressionModel, IsotonicRegression]( - ir, isClassification = false, sqlContext) { (expected, actual) => + ir, spark, isClassification = false) { (expected, actual) => assert(expected.boundaries === actual.boundaries) assert(expected.predictions === actual.predictions) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index eb19d130939e..265f2f45c45f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -21,12 +21,12 @@ import scala.util.Random import org.apache.spark.SparkFunSuite import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{DenseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{DenseVector, Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.{LinearDataGenerator, MLlibTestSparkContext} -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} class LinearRegressionSuite @@ -42,29 +42,29 @@ class LinearRegressionSuite override def beforeAll(): Unit = { super.beforeAll() - datasetWithDenseFeature = sqlContext.createDataFrame( + datasetWithDenseFeature = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( intercept = 6.3, weights = Array(4.7, 7.2), xMean = Array(0.9, -1.3), - xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2)) + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2).map(_.asML)) /* datasetWithDenseFeatureWithoutIntercept is not needed for correctness testing but is useful for illustrating training model without intercept */ - datasetWithDenseFeatureWithoutIntercept = sqlContext.createDataFrame( + datasetWithDenseFeatureWithoutIntercept = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( intercept = 0.0, weights = Array(4.7, 7.2), xMean = Array(0.9, -1.3), - xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2)) + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2).map(_.asML)) val r = new Random(seed) // When feature size is larger than 4096, normal optimizer is choosed // as the solver of linear regression in the case of "auto" mode. val featureSize = 4100 - datasetWithSparseFeature = sqlContext.createDataFrame( + datasetWithSparseFeature = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( intercept = 0.0, weights = Seq.fill(featureSize)(r.nextDouble()).toArray, xMean = Seq.fill(featureSize)(r.nextDouble()).toArray, xVariance = Seq.fill(featureSize)(r.nextDouble()).toArray, nPoints = 200, - seed, eps = 0.1, sparsity = 0.7), 2)) + seed, eps = 0.1, sparsity = 0.7), 2).map(_.asML)) /* R code: @@ -74,7 +74,7 @@ class LinearRegressionSuite w <- c(1, 2, 3, 4) df <- as.data.frame(cbind(A, b)) */ - datasetWithWeight = sqlContext.createDataFrame( + datasetWithWeight = spark.createDataFrame( sc.parallelize(Seq( Instance(17.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), Instance(19.0, 2.0, Vectors.dense(1.0, 7.0)), @@ -90,14 +90,14 @@ class LinearRegressionSuite w <- c(1, 2, 3, 4) df.const.label <- as.data.frame(cbind(A, b.const)) */ - datasetWithWeightConstantLabel = sqlContext.createDataFrame( + datasetWithWeightConstantLabel = spark.createDataFrame( sc.parallelize(Seq( Instance(17.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), Instance(17.0, 2.0, Vectors.dense(1.0, 7.0)), Instance(17.0, 3.0, Vectors.dense(2.0, 11.0)), Instance(17.0, 4.0, Vectors.dense(3.0, 13.0)) ), 2)) - datasetWithWeightZeroLabel = sqlContext.createDataFrame( + datasetWithWeightZeroLabel = spark.createDataFrame( sc.parallelize(Seq( Instance(0.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), Instance(0.0, 2.0, Vectors.dense(1.0, 7.0)), @@ -610,20 +610,31 @@ class LinearRegressionSuite val model1 = new LinearRegression() .setFitIntercept(fitIntercept) .setWeightCol("weight") + .setPredictionCol("myPrediction") .setSolver(solver) .fit(datasetWithWeightConstantLabel) val actual1 = Vectors.dense(model1.intercept, model1.coefficients(0), model1.coefficients(1)) assert(actual1 ~== expected(idx) absTol 1e-4) + // Schema of summary.predictions should be a superset of the input dataset + assert((datasetWithWeightConstantLabel.schema.fieldNames.toSet + model1.getPredictionCol) + .subsetOf(model1.summary.predictions.schema.fieldNames.toSet)) + val model2 = new LinearRegression() .setFitIntercept(fitIntercept) .setWeightCol("weight") + .setPredictionCol("myPrediction") .setSolver(solver) .fit(datasetWithWeightZeroLabel) val actual2 = Vectors.dense(model2.intercept, model2.coefficients(0), model2.coefficients(1)) assert(actual2 ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1e-4) + + // Schema of summary.predictions should be a superset of the input dataset + assert((datasetWithWeightZeroLabel.schema.fieldNames.toSet + model2.getPredictionCol) + .subsetOf(model2.summary.predictions.schema.fieldNames.toSet)) + idx += 1 } } @@ -672,7 +683,7 @@ class LinearRegressionSuite test("linear regression model training summary") { Seq("auto", "l-bfgs", "normal").foreach { solver => - val trainer = new LinearRegression().setSolver(solver) + val trainer = new LinearRegression().setSolver(solver).setPredictionCol("myPrediction") val model = trainer.fit(datasetWithDenseFeature) val trainerNoPredictionCol = trainer.setPredictionCol("") val modelNoPredictionCol = trainerNoPredictionCol.fit(datasetWithDenseFeature) @@ -682,7 +693,7 @@ class LinearRegressionSuite assert(modelNoPredictionCol.hasSummary) // Schema should be a superset of the input dataset - assert((datasetWithDenseFeature.schema.fieldNames.toSet + "prediction").subsetOf( + assert((datasetWithDenseFeature.schema.fieldNames.toSet + model.getPredictionCol).subsetOf( model.summary.predictions.schema.fieldNames.toSet)) // Validate that we re-insert a prediction column for evaluation val modelNoPredictionColFieldNames @@ -795,7 +806,7 @@ class LinearRegressionSuite Seq("auto", "l-bfgs", "normal").foreach { solver => val (data, weightedData) = { val activeData = LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 500, 1, 0.1) + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 500, 1, 0.1).map(_.asML) val rnd = new Random(8392) val signedData = activeData.map { case p: LabeledPoint => @@ -822,14 +833,14 @@ class LinearRegressionSuite } val noiseData = LinearDataGenerator.generateLinearInput( - 2, Array(1, 3), Array(0.9, -1.3), Array(0.7, 1.2), 500, 1, 0.1) + 2, Array(1, 3), Array(0.9, -1.3), Array(0.7, 1.2), 500, 1, 0.1).map(_.asML) val weightedNoiseData = noiseData.map { case LabeledPoint(label, features) => Instance(label, weight = 0, features) } val data2 = weightedSignedData ++ weightedNoiseData - (sqlContext.createDataFrame(sc.parallelize(data1, 4)), - sqlContext.createDataFrame(sc.parallelize(data2, 4))) + (spark.createDataFrame(sc.parallelize(data1, 4)), + spark.createDataFrame(sc.parallelize(data2, 4))) } val trainer1a = (new LinearRegression).setFitIntercept(true) @@ -1010,7 +1021,7 @@ class LinearRegressionSuite test("should support all NumericType labels and not support other types") { val lr = new LinearRegression().setMaxIter(1) MLTestingUtils.checkNumericTypes[LinearRegressionModel, LinearRegression]( - lr, isClassification = false, sqlContext) { (expected, actual) => + lr, spark, isClassification = false) { (expected, actual) => assert(expected.intercept === actual.intercept) assert(expected.coefficients === actual.coefficients) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala index ca400e191451..c08335f9f84a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala @@ -18,9 +18,10 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, RandomForest => OldRandomForest} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -40,7 +41,8 @@ class RandomForestRegressorSuite extends SparkFunSuite with MLlibTestSparkContex override def beforeAll() { super.beforeAll() orderedLabeledPoints50_1000 = - sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)) + sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) + .map(_.asML)) } ///////////////////////////////////////////////////////////////////////////// @@ -98,7 +100,7 @@ class RandomForestRegressorSuite extends SparkFunSuite with MLlibTestSparkContex test("should support all NumericType labels and not support other types") { val rf = new RandomForestRegressor().setMaxDepth(1) MLTestingUtils.checkNumericTypes[RandomForestRegressionModel, RandomForestRegressor]( - rf, isClassification = false, sqlContext) { (expected, actual) => + rf, spark, isClassification = false) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } @@ -139,8 +141,8 @@ private object RandomForestRegressorSuite extends SparkFunSuite { val numFeatures = data.first().features.size val oldStrategy = rf.getOldStrategy(categoricalFeatures, numClasses = 0, OldAlgo.Regression, rf.getOldImpurity) - val oldModel = OldRandomForest.trainRegressor( - data, oldStrategy, rf.getNumTrees, rf.getFeatureSubsetStrategy, rf.getSeed.toInt) + val oldModel = OldRandomForest.trainRegressor(data.map(OldLabeledPoint.fromML), oldStrategy, + rf.getNumTrees, rf.getFeatureSubsetStrategy, rf.getSeed.toInt) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) val newModel = rf.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala index e52fbd74a7b4..2517de59fed6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala @@ -23,7 +23,7 @@ import java.nio.charset.StandardCharsets import com.google.common.io.Files import org.apache.spark.{SparkException, SparkFunSuite} -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{Row, SaveMode} import org.apache.spark.util.Utils @@ -56,7 +56,7 @@ class LibSVMRelationSuite extends SparkFunSuite with MLlibTestSparkContext { } test("select as sparse vector") { - val df = sqlContext.read.format("libsvm").load(path) + val df = spark.read.format("libsvm").load(path) assert(df.columns(0) == "label") assert(df.columns(1) == "features") val row1 = df.first() @@ -66,7 +66,7 @@ class LibSVMRelationSuite extends SparkFunSuite with MLlibTestSparkContext { } test("select as dense vector") { - val df = sqlContext.read.format("libsvm").options(Map("vectorType" -> "dense")) + val df = spark.read.format("libsvm").options(Map("vectorType" -> "dense")) .load(path) assert(df.columns(0) == "label") assert(df.columns(1) == "features") @@ -78,7 +78,7 @@ class LibSVMRelationSuite extends SparkFunSuite with MLlibTestSparkContext { } test("select a vector with specifying the longer dimension") { - val df = sqlContext.read.option("numFeatures", "100").format("libsvm") + val df = spark.read.option("numFeatures", "100").format("libsvm") .load(path) val row1 = df.first() val v = row1.getAs[SparseVector](1) @@ -86,27 +86,28 @@ class LibSVMRelationSuite extends SparkFunSuite with MLlibTestSparkContext { } test("write libsvm data and read it again") { - val df = sqlContext.read.format("libsvm").load(path) + val df = spark.read.format("libsvm").load(path) val tempDir2 = new File(tempDir, "read_write_test") val writepath = tempDir2.toURI.toString // TODO: Remove requirement to coalesce by supporting multiple reads. df.coalesce(1).write.format("libsvm").mode(SaveMode.Overwrite).save(writepath) - val df2 = sqlContext.read.format("libsvm").load(writepath) + val df2 = spark.read.format("libsvm").load(writepath) val row1 = df2.first() val v = row1.getAs[SparseVector](1) assert(v == Vectors.sparse(6, Seq((0, 1.0), (2, 2.0), (4, 3.0)))) } test("write libsvm data failed due to invalid schema") { - val df = sqlContext.read.format("text").load(path) + val df = spark.read.format("text").load(path) intercept[SparkException] { df.write.format("libsvm").save(path + "_2") } } test("select features from libsvm relation") { - val df = sqlContext.read.format("libsvm").load(path) + val df = spark.read.format("libsvm").load(path) df.select("features").rdd.map { case Row(d: Vector) => d }.first + df.select("features").collect } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala index fecf372c3d84..5c50a88c8314 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.tree.impl import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.mllib.tree.{GradientBoostedTreesSuite => OldGBTSuite} import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ @@ -35,10 +35,10 @@ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext test("runWithValidation stops early and performs better on a validation dataset") { // Set numIterations large enough so that it stops early. val numIterations = 20 - val trainRdd = sc.parallelize(OldGBTSuite.trainData, 2) - val validateRdd = sc.parallelize(OldGBTSuite.validateData, 2) - val trainDF = sqlContext.createDataFrame(trainRdd) - val validateDF = sqlContext.createDataFrame(validateRdd) + val trainRdd = sc.parallelize(OldGBTSuite.trainData, 2).map(_.asML) + val validateRdd = sc.parallelize(OldGBTSuite.validateData, 2).map(_.asML) + val trainDF = spark.createDataFrame(trainRdd) + val validateDF = spark.createDataFrame(validateRdd) val algos = Array(Regression, Regression, Classification) val losses = Array(SquaredError, AbsoluteError, LogLoss) diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala index 9739e6c05dcb..dcc2f305df75 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala @@ -21,14 +21,14 @@ import scala.collection.mutable import org.apache.spark.SparkFunSuite import org.apache.spark.ml.classification.DecisionTreeClassificationModel +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.tree._ -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.tree.{DecisionTreeSuite => OldDTSuite, EnsembleTestHelper} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, QuantileStrategy, Strategy => OldStrategy} import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, GiniCalculator} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.collection.OpenHashMap /** @@ -43,7 +43,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { ///////////////////////////////////////////////////////////////////////////// test("Binary classification with continuous features: split calculation") { - val arr = OldDTSuite.generateOrderedLabeledPointsWithLabel1() + val arr = OldDTSuite.generateOrderedLabeledPointsWithLabel1().map(_.asML) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, 3, 2, 100) @@ -55,7 +55,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Binary classification with binary (ordered) categorical features: split calculation") { - val arr = OldDTSuite.generateCategoricalDataPoints() + val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, maxDepth = 2, numClasses = 2, @@ -72,7 +72,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { test("Binary classification with 3-ary (ordered) categorical features," + " with no samples for one category: split calculation") { - val arr = OldDTSuite.generateCategoricalDataPoints() + val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, maxDepth = 2, numClasses = 2, @@ -148,7 +148,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Multiclass classification with unordered categorical features: split calculations") { - val arr = OldDTSuite.generateCategoricalDataPoints() + val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy( @@ -189,7 +189,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Multiclass classification with ordered categorical features: split calculations") { - val arr = OldDTSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() + val arr = OldDTSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures().map(_.asML) assert(arr.length === 3000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, maxDepth = 2, numClasses = 100, @@ -334,7 +334,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Second level node building with vs. without groups") { - val arr = OldDTSuite.generateOrderedLabeledPoints() + val arr = OldDTSuite.generateOrderedLabeledPoints().map(_.asML) assert(arr.length === 1000) val rdd = sc.parallelize(arr) // For tree with 1 group @@ -378,7 +378,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { def binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy: OldStrategy) { val numFeatures = 50 val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures, 1000) - val rdd = sc.parallelize(arr) + val rdd = sc.parallelize(arr).map(_.asML) // Select feature subset for top nodes. Return true if OK. def checkFeatureSubsetStrategy( diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala index e3f09899d769..8cbd652bacf3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala @@ -22,11 +22,11 @@ import scala.collection.JavaConverters._ import org.apache.spark.{SparkContext, SparkFunSuite} import org.apache.spark.api.java.JavaRDD import org.apache.spark.ml.attribute.{AttributeGroup, NominalAttribute, NumericAttribute} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.tree._ -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, SparkSession} private[ml] object TreeTests extends SparkFunSuite { @@ -42,8 +42,12 @@ private[ml] object TreeTests extends SparkFunSuite { data: RDD[LabeledPoint], categoricalFeatures: Map[Int, Int], numClasses: Int): DataFrame = { - val sqlContext = SQLContext.getOrCreate(data.sparkContext) - import sqlContext.implicits._ + val spark = SparkSession.builder + .master("local[2]") + .appName("TreeTests") + .getOrCreate() + import spark.implicits._ + val df = data.toDF() val numFeatures = data.first().features.size val featuresAttributes = Range(0, numFeatures).map { feature => diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala index 3e734aabc554..30bd390381e9 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala @@ -20,17 +20,17 @@ package org.apache.spark.ml.tuning import org.apache.spark.SparkFunSuite import org.apache.spark.ml.{Estimator, Model, Pipeline} import org.apache.spark.ml.classification.{LogisticRegression, LogisticRegressionModel} +import org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInput import org.apache.spark.ml.evaluation.{BinaryClassificationEvaluator, Evaluator, RegressionEvaluator} import org.apache.spark.ml.feature.HashingTF +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.{ParamMap, ParamPair} import org.apache.spark.ml.param.shared.HasInputCol import org.apache.spark.ml.regression.LinearRegression import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{LinearDataGenerator, MLlibTestSparkContext} -import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.types.StructType class CrossValidatorSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { @@ -39,7 +39,7 @@ class CrossValidatorSuite override def beforeAll(): Unit = { super.beforeAll() - dataset = sqlContext.createDataFrame( + dataset = spark.createDataFrame( sc.parallelize(generateLogisticInput(1.0, 1.0, 100, 42), 2)) } @@ -67,9 +67,9 @@ class CrossValidatorSuite } test("cross validation with linear regression") { - val dataset = sqlContext.createDataFrame( + val dataset = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2)) + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2).map(_.asML)) val trainer = new LinearRegression().setSolver("l-bfgs") val lrParamMaps = new ParamGridBuilder() @@ -136,6 +136,7 @@ class CrossValidatorSuite assert(cv.uid === cv2.uid) assert(cv.getNumFolds === cv2.getNumFolds) + assert(cv.getSeed === cv2.getSeed) assert(cv2.getEvaluator.isInstanceOf[BinaryClassificationEvaluator]) val evaluator2 = cv2.getEvaluator.asInstanceOf[BinaryClassificationEvaluator] @@ -186,6 +187,7 @@ class CrossValidatorSuite assert(cv.uid === cv2.uid) assert(cv.getNumFolds === cv2.getNumFolds) + assert(cv.getSeed === cv2.getSeed) assert(cv2.getEvaluator.isInstanceOf[BinaryClassificationEvaluator]) assert(cv.getEvaluator.uid === cv2.getEvaluator.uid) @@ -259,6 +261,7 @@ class CrossValidatorSuite assert(cv.uid === cv2.uid) assert(cv.getNumFolds === cv2.getNumFolds) + assert(cv.getSeed === cv2.getSeed) assert(cv2.getEvaluator.isInstanceOf[BinaryClassificationEvaluator]) val evaluator2 = cv2.getEvaluator.asInstanceOf[BinaryClassificationEvaluator] diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala index dbee47c8475d..c1e9c2fc1dc1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala @@ -20,21 +20,21 @@ package org.apache.spark.ml.tuning import org.apache.spark.SparkFunSuite import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.classification.{LogisticRegression, LogisticRegressionModel} +import org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInput import org.apache.spark.ml.evaluation.{BinaryClassificationEvaluator, Evaluator, RegressionEvaluator} +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared.HasInputCol import org.apache.spark.ml.regression.LinearRegression import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{LinearDataGenerator, MLlibTestSparkContext} -import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.Dataset import org.apache.spark.sql.types.StructType class TrainValidationSplitSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { test("train validation with logistic regression") { - val dataset = sqlContext.createDataFrame( + val dataset = spark.createDataFrame( sc.parallelize(generateLogisticInput(1.0, 1.0, 100, 42), 2)) val lr = new LogisticRegression @@ -58,9 +58,9 @@ class TrainValidationSplitSuite } test("train validation with linear regression") { - val dataset = sqlContext.createDataFrame( + val dataset = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2)) + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2).map(_.asML)) val trainer = new LinearRegression().setSolver("l-bfgs") val lrParamMaps = new ParamGridBuilder() @@ -127,6 +127,7 @@ class TrainValidationSplitSuite val tvs2 = testDefaultReadWrite(tvs, testParams = false) assert(tvs.getTrainRatio === tvs2.getTrainRatio) + assert(tvs.getSeed === tvs2.getSeed) } test("read/write: TrainValidationSplitModel") { @@ -149,6 +150,7 @@ class TrainValidationSplitSuite assert(tvs.getTrainRatio === tvs2.getTrainRatio) assert(tvs.validationMetrics === tvs2.validationMetrics) + assert(tvs.getSeed === tvs2.getSeed) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala index d9e6fd5aae67..80b976914cbd 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala @@ -20,10 +20,11 @@ package org.apache.spark.ml.util import org.apache.spark.SparkFunSuite import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.recommendation.{ALS, ALSModel} import org.apache.spark.ml.tree.impl.TreeTests -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -37,18 +38,18 @@ object MLTestingUtils extends SparkFunSuite { def checkNumericTypes[M <: Model[M], T <: Estimator[M]]( estimator: T, - isClassification: Boolean, - sqlContext: SQLContext)(check: (M, M) => Unit): Unit = { + spark: SparkSession, + isClassification: Boolean = true)(check: (M, M) => Unit): Unit = { val dfs = if (isClassification) { - genClassifDFWithNumericLabelCol(sqlContext) + genClassifDFWithNumericLabelCol(spark) } else { - genRegressionDFWithNumericLabelCol(sqlContext) + genRegressionDFWithNumericLabelCol(spark) } val expected = estimator.fit(dfs(DoubleType)) val actuals = dfs.keys.filter(_ != DoubleType).map(t => estimator.fit(dfs(t))) actuals.foreach(actual => check(expected, actual)) - val dfWithStringLabels = sqlContext.createDataFrame(Seq( + val dfWithStringLabels = spark.createDataFrame(Seq( ("0", Vectors.dense(0, 2, 3), 0.0) )).toDF("label", "features", "censor") val thrown = intercept[IllegalArgumentException] { @@ -58,13 +59,37 @@ object MLTestingUtils extends SparkFunSuite { "Column label must be of type NumericType but was actually of type StringType")) } - def checkNumericTypes[T <: Evaluator](evaluator: T, sqlContext: SQLContext): Unit = { - val dfs = genEvaluatorDFWithNumericLabelCol(sqlContext, "label", "prediction") + def checkNumericTypesALS( + estimator: ALS, + spark: SparkSession, + column: String, + baseType: NumericType) + (check: (ALSModel, ALSModel) => Unit) + (check2: (ALSModel, ALSModel, DataFrame) => Unit): Unit = { + val dfs = genRatingsDFWithNumericCols(spark, column) + val expected = estimator.fit(dfs(baseType)) + val actuals = dfs.keys.filter(_ != baseType).map(t => (t, estimator.fit(dfs(t)))) + actuals.foreach { case (_, actual) => check(expected, actual) } + actuals.foreach { case (t, actual) => check2(expected, actual, dfs(t)) } + + val baseDF = dfs(baseType) + val others = baseDF.columns.toSeq.diff(Seq(column)).map(col(_)) + val cols = Seq(col(column).cast(StringType)) ++ others + val strDF = baseDF.select(cols: _*) + val thrown = intercept[IllegalArgumentException] { + estimator.fit(strDF) + } + assert(thrown.getMessage.contains( + s"$column must be of type NumericType but was actually of type StringType")) + } + + def checkNumericTypes[T <: Evaluator](evaluator: T, spark: SparkSession): Unit = { + val dfs = genEvaluatorDFWithNumericLabelCol(spark, "label", "prediction") val expected = evaluator.evaluate(dfs(DoubleType)) val actuals = dfs.keys.filter(_ != DoubleType).map(t => evaluator.evaluate(dfs(t))) actuals.foreach(actual => assert(expected === actual)) - val dfWithStringLabels = sqlContext.createDataFrame(Seq( + val dfWithStringLabels = spark.createDataFrame(Seq( ("0", 0d) )).toDF("label", "prediction") val thrown = intercept[IllegalArgumentException] { @@ -75,10 +100,10 @@ object MLTestingUtils extends SparkFunSuite { } def genClassifDFWithNumericLabelCol( - sqlContext: SQLContext, + spark: SparkSession, labelColName: String = "label", featuresColName: String = "features"): Map[NumericType, DataFrame] = { - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, Vectors.dense(0, 2, 3)), (1, Vectors.dense(0, 3, 1)), (0, Vectors.dense(0, 2, 2)), @@ -95,11 +120,11 @@ object MLTestingUtils extends SparkFunSuite { } def genRegressionDFWithNumericLabelCol( - sqlContext: SQLContext, + spark: SparkSession, labelColName: String = "label", featuresColName: String = "features", censorColName: String = "censor"): Map[NumericType, DataFrame] = { - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, Vectors.dense(0)), (1, Vectors.dense(1)), (2, Vectors.dense(2)), @@ -116,11 +141,31 @@ object MLTestingUtils extends SparkFunSuite { }.toMap } + def genRatingsDFWithNumericCols( + spark: SparkSession, + column: String): Map[NumericType, DataFrame] = { + val df = spark.createDataFrame(Seq( + (0, 10, 1.0), + (1, 20, 2.0), + (2, 30, 3.0), + (3, 40, 4.0), + (4, 50, 5.0) + )).toDF("user", "item", "rating") + + val others = df.columns.toSeq.diff(Seq(column)).map(col(_)) + val types: Seq[NumericType] = + Seq(ShortType, LongType, IntegerType, FloatType, ByteType, DoubleType, DecimalType(10, 0)) + types.map { t => + val cols = Seq(col(column).cast(t)) ++ others + t -> df.select(cols: _*) + }.toMap + } + def genEvaluatorDFWithNumericLabelCol( - sqlContext: SQLContext, + spark: SparkSession, labelColName: String = "label", predictionColName: String = "prediction"): Map[NumericType, DataFrame] = { - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, 0d), (1, 1d), (2, 2d), diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala index 9e6bc7193c13..141249a427a4 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala @@ -60,9 +60,9 @@ class StopwatchSuite extends SparkFunSuite with MLlibTestSparkContext { test("DistributedStopwatch on executors") { val sw = new DistributedStopwatch(sc, "sw") val rdd = sc.parallelize(0 until 4, 4) - val acc = sc.accumulator(0L) + val acc = sc.longAccumulator rdd.foreach { i => - acc += checkStopwatch(sw) + acc.add(checkStopwatch(sw)) } assert(!sw.isRunning) val elapsed = sw.elapsed() @@ -88,12 +88,12 @@ class StopwatchSuite extends SparkFunSuite with MLlibTestSparkContext { assert(sw.toString === s"{\n local: ${localElapsed}ms,\n spark: ${sparkElapsed}ms\n}") val rdd = sc.parallelize(0 until 4, 4) - val acc = sc.accumulator(0L) + val acc = sc.longAccumulator rdd.foreach { i => sw("local").start() val duration = checkStopwatch(sw("spark")) sw("local").stop() - acc += duration + acc.add(duration) } val localElapsed2 = sw("local").elapsed() assert(localElapsed2 === localElapsed) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala index d55bc8c3ec09..f316c67234f1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala @@ -69,11 +69,12 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { assert(math.abs(metrics.fMeasure(1.0, 2.0) - f2measure1) < delta) assert(math.abs(metrics.fMeasure(2.0, 2.0) - f2measure2) < delta) - assert(math.abs(metrics.recall - + assert(math.abs(metrics.accuracy - (2.0 + 3.0 + 1.0) / ((2 + 3 + 1) + (1 + 1 + 1))) < delta) - assert(math.abs(metrics.recall - metrics.precision) < delta) - assert(math.abs(metrics.recall - metrics.fMeasure) < delta) - assert(math.abs(metrics.recall - metrics.weightedRecall) < delta) + assert(math.abs(metrics.accuracy - metrics.precision) < delta) + assert(math.abs(metrics.accuracy - metrics.recall) < delta) + assert(math.abs(metrics.accuracy - metrics.fMeasure) < delta) + assert(math.abs(metrics.accuracy - metrics.weightedRecall) < delta) assert(math.abs(metrics.weightedFalsePositiveRate - ((4.0 / 9) * fpRate0 + (4.0 / 9) * fpRate1 + (1.0 / 9) * fpRate2)) < delta) assert(math.abs(metrics.weightedPrecision - diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index b7df02e6c098..8c5b4bda2518 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -560,4 +560,55 @@ class MatricesSuite extends SparkFunSuite { compare(oldSM0, newSM0) compare(oldDM0, newDM0) } + + test("implicit conversions between new local linalg and mllib linalg") { + + def mllibMatrixToTriple(m: Matrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mllibDenseMatrixToTriple(m: DenseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mllibSparseMatrixToTriple(m: SparseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mlMatrixToTriple(m: newlinalg.Matrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mlDenseMatrixToTriple(m: newlinalg.DenseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mlSparseMatrixToTriple(m: newlinalg.SparseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def compare(m1: (Array[Double], Int, Int), m2: (Array[Double], Int, Int)): Unit = { + assert(m1._1 === m2._1) + assert(m1._2 === m2._2) + assert(m1._3 === m2._3) + } + + val dm: DenseMatrix = new DenseMatrix(3, 2, Array(0.0, 0.0, 1.0, 0.0, 2.0, 3.5)) + val sm: SparseMatrix = dm.toSparse + val sm0: Matrix = sm.asInstanceOf[Matrix] + val dm0: Matrix = dm.asInstanceOf[Matrix] + + val newSM: newlinalg.SparseMatrix = sm.asML + val newDM: newlinalg.DenseMatrix = dm.asML + val newSM0: newlinalg.Matrix = sm0.asML + val newDM0: newlinalg.Matrix = dm0.asML + + import org.apache.spark.mllib.linalg.MatrixImplicits._ + + compare(mllibMatrixToTriple(dm0), mllibMatrixToTriple(newDM0)) + compare(mllibMatrixToTriple(sm0), mllibMatrixToTriple(newSM0)) + + compare(mllibDenseMatrixToTriple(dm), mllibDenseMatrixToTriple(newDM)) + compare(mllibSparseMatrixToTriple(sm), mllibSparseMatrixToTriple(newSM)) + + compare(mlMatrixToTriple(dm0), mlMatrixToTriple(newDM)) + compare(mlMatrixToTriple(sm0), mlMatrixToTriple(newSM0)) + + compare(mlDenseMatrixToTriple(dm), mlDenseMatrixToTriple(newDM)) + compare(mlSparseMatrixToTriple(sm), mlSparseMatrixToTriple(newSM)) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index a7c1a076044e..2e9c40ab88ed 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -422,4 +422,43 @@ class VectorsSuite extends SparkFunSuite with Logging { assert(oldSV0.toArray === newSV0.toArray) assert(oldDV0.toArray === newDV0.toArray) } + + test("implicit conversions between new local linalg and mllib linalg") { + + def mllibVectorToArray(v: Vector): Array[Double] = v.toArray + + def mllibDenseVectorToArray(v: DenseVector): Array[Double] = v.toArray + + def mllibSparseVectorToArray(v: SparseVector): Array[Double] = v.toArray + + def mlVectorToArray(v: newlinalg.Vector): Array[Double] = v.toArray + + def mlDenseVectorToArray(v: newlinalg.DenseVector): Array[Double] = v.toArray + + def mlSparseVectorToArray(v: newlinalg.SparseVector): Array[Double] = v.toArray + + val dv: DenseVector = new DenseVector(Array(1.0, 2.0, 3.5)) + val sv: SparseVector = new SparseVector(5, Array(1, 2, 4), Array(1.1, 2.2, 4.4)) + val sv0: Vector = sv.asInstanceOf[Vector] + val dv0: Vector = dv.asInstanceOf[Vector] + + val newSV: newlinalg.SparseVector = sv.asML + val newDV: newlinalg.DenseVector = dv.asML + val newSV0: newlinalg.Vector = sv0.asML + val newDV0: newlinalg.Vector = dv0.asML + + import org.apache.spark.mllib.linalg.VectorImplicits._ + + assert(mllibVectorToArray(dv0) === mllibVectorToArray(newDV0)) + assert(mllibVectorToArray(sv0) === mllibVectorToArray(newSV0)) + + assert(mllibDenseVectorToArray(dv) === mllibDenseVectorToArray(newDV)) + assert(mllibSparseVectorToArray(sv) === mllibSparseVectorToArray(newSV)) + + assert(mlVectorToArray(dv0) === mlVectorToArray(newDV0)) + assert(mlVectorToArray(sv0) === mlVectorToArray(newSV0)) + + assert(mlDenseVectorToArray(dv) === mlDenseVectorToArray(newDV)) + assert(mlSparseVectorToArray(sv) === mlSparseVectorToArray(newSV)) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala index f8d0af8820e6..252a068dcd72 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} import org.apache.spark.mllib.linalg.Vectors class LabeledPointSuite extends SparkFunSuite { @@ -40,4 +41,16 @@ class LabeledPointSuite extends SparkFunSuite { val point = LabeledPoint.parse("1.0,1.0 0.0 -2.0") assert(point === LabeledPoint(1.0, Vectors.dense(1.0, 0.0, -2.0))) } + + test("conversions between new ml LabeledPoint and mllib LabeledPoint") { + val points: Seq[LabeledPoint] = Seq( + LabeledPoint(1.0, Vectors.dense(1.0, 0.0)), + LabeledPoint(0.0, Vectors.sparse(2, Array(1), Array(-1.0)))) + + val newPoints: Seq[NewLabeledPoint] = points.map(_.asML) + + points.zip(newPoints).foreach { case (p1, p2) => + assert(p1 === LabeledPoint.fromML(p2)) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 49cb7e1f24e3..441d0f7614bf 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -73,7 +73,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -100,7 +100,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1 -> 2)) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -116,7 +116,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -133,7 +133,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -150,7 +150,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -167,7 +167,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -183,7 +183,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClasses = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(strategy.isMulticlassClassification) assert(metadata.isUnordered(featureIndex = 0)) assert(metadata.isUnordered(featureIndex = 1)) @@ -240,7 +240,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { numClasses = 3, maxBins = maxBins, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(metadata.isUnordered(featureIndex = 0)) assert(metadata.isUnordered(featureIndex = 1)) @@ -288,7 +288,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClasses = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(metadata.isUnordered(featureIndex = 0)) val model = DecisionTree.train(rdd, strategy) @@ -310,7 +310,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { numClasses = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala index 7f9e340f54b6..ba8d36f45f55 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala @@ -23,23 +23,22 @@ import org.scalatest.Suite import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.util.TempDirectory -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.util.Utils trait MLlibTestSparkContext extends TempDirectory { self: Suite => + @transient var spark: SparkSession = _ @transient var sc: SparkContext = _ - @transient var sqlContext: SQLContext = _ @transient var checkpointDir: String = _ override def beforeAll() { super.beforeAll() - val conf = new SparkConf() - .setMaster("local[2]") - .setAppName("MLlibUnitTest") - sc = new SparkContext(conf) - SQLContext.clearActive() - sqlContext = new SQLContext(sc) - SQLContext.setActive(sqlContext) + spark = SparkSession.builder + .master("local[2]") + .appName("MLlibUnitTest") + .getOrCreate() + sc = spark.sparkContext + checkpointDir = Utils.createDirectory(tempDir.getCanonicalPath, "checkpoints").toString sc.setCheckpointDir(checkpointDir) } @@ -47,12 +46,11 @@ trait MLlibTestSparkContext extends TempDirectory { self: Suite => override def afterAll() { try { Utils.deleteRecursively(new File(checkpointDir)) - sqlContext = null SQLContext.clearActive() - if (sc != null) { - sc.stop() + if (spark != null) { + spark.stop() } - sc = null + spark = null } finally { super.afterAll() } diff --git a/pom.xml b/pom.xml index 66f1d8ea902a..9c13af17e4ed 100644 --- a/pom.xml +++ b/pom.xml @@ -101,7 +101,6 @@ sql/core sql/hive sql/hivecontext-compatibility - external/docker-integration-tests assembly external/flume external/flume-sink @@ -109,8 +108,8 @@ examples repl launcher - external/kafka - external/kafka-assembly + external/kafka-0-8 + external/kafka-0-8-assembly @@ -134,11 +133,11 @@ 1.2.1.spark2 1.2.1 - 10.10.1.1 + 10.11.1.1 1.7.0 1.6.0 - 8.1.14.v20131031 - 3.0.0.v201112011016 + 9.2.16.v20160414 + 3.1.0 0.8.0 2.4.0 2.0.8 @@ -150,8 +149,8 @@ 0.10.2 - 4.3.2 - 4.3.2 + 4.5.2 + 4.4.4 3.1 3.4.1 @@ -167,19 +166,21 @@ 1.1.2 1.2.0-incubating 1.10 + 2.4 2.6 3.3.2 3.2.10 2.7.8 - 1.9 + 2.22.2 2.9.3 3.5.2 1.3.9 0.9.2 4.5.2-1 1.1 + 2.52.0 ${java.home} @@ -285,9 +286,8 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} ${project.version} - test com.twitter @@ -329,6 +329,12 @@ ${jetty.version} provided + + org.eclipse.jetty + jetty-servlets + ${jetty.version} + provided + org.eclipse.jetty jetty-util @@ -360,7 +366,6 @@ provided - org.apache.commons commons-lang3 @@ -371,6 +376,11 @@ commons-lang ${commons-lang2.version} + + commons-io + commons-io + ${commons-io.version} + commons-codec commons-codec @@ -406,6 +416,11 @@ httpclient ${commons.httpclient.version} + + org.apache.httpcomponents + httpmime + ${commons.httpclient.version} + org.apache.httpcomponents httpcore @@ -414,7 +429,7 @@ org.seleniumhq.selenium selenium-java - 2.45.0 + ${selenium.version} test @@ -427,6 +442,12 @@ + + org.seleniumhq.selenium + selenium-htmlunit-driver + ${selenium.version} + test + xml-apis @@ -588,16 +609,44 @@ - com.sun.jersey + com.fasterxml.jackson.module + jackson-module-jaxb-annotations + ${fasterxml.jackson.version} + + + org.glassfish.jersey.core jersey-server ${jersey.version} - ${hadoop.deps.scope} - com.sun.jersey - jersey-core + org.glassfish.jersey.core + jersey-common + ${jersey.version} + + + org.glassfish.jersey.core + jersey-client ${jersey.version} - ${hadoop.deps.scope} + + + org.glassfish.jersey.containers + jersey-container-servlet + ${jersey.version} + + + org.glassfish.jersey.containers + jersey-container-servlet-core + ${jersey.version} + + + org.glassfish.jersey + jersey-client + ${jersey.version} + + + javax.ws.rs + javax.ws.rs-api + 2.0.1 org.scalanlp @@ -619,18 +668,7 @@ org.json4s json4s-jackson_${scala.binary.version} - 3.2.10 - - - com.sun.jersey - jersey-json - ${jersey.version} - - - stax - stax-api - - + 3.2.11 org.scala-lang @@ -710,18 +748,6 @@ guava com.google.guava - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - - - commons-logging - httpclient - commons-logging commons-logging @@ -814,6 +840,18 @@ junit junit + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -926,6 +964,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -954,6 +1004,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -983,6 +1045,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -1011,6 +1085,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -1039,6 +1125,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -1355,14 +1453,6 @@ ${hive.group} hive-shims - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.apache.curator curator-framework @@ -1714,14 +1804,6 @@ libthrift ${libthrift.version} - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.slf4j slf4j-api @@ -1733,14 +1815,6 @@ libfb303 ${libthrift.version} - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.slf4j slf4j-api @@ -2177,6 +2251,7 @@ org.eclipse.jetty:jetty-http org.eclipse.jetty:jetty-continuation org.eclipse.jetty:jetty-servlet + org.eclipse.jetty:jetty-servlets org.eclipse.jetty:jetty-plus org.eclipse.jetty:jetty-security org.eclipse.jetty:jetty-util @@ -2381,6 +2456,13 @@ + + docker-integration-tests + + external/docker-integration-tests + + + org.seleniumhq.selenium selenium-java test + + org.seleniumhq.selenium + selenium-htmlunit-driver + test + org.apache.spark spark-sql_${scala.binary.version} @@ -91,7 +84,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} net.sf.jpam @@ -118,12 +111,6 @@ - - - - org.codehaus.mojo - build-helper-maven-plugin - add-source generate-sources diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java index 2111837cac8d..b95077cd6218 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java @@ -29,7 +29,7 @@ public interface Service { /** * Service states */ - public enum STATE { + enum STATE { /** Constructed but not initialized */ NOTINITED, diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java index e712aaf2348f..edb5eff9615b 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java @@ -41,4 +41,4 @@ public static int indexOfDomainMatch(String userName) { } return endIdx; } -} \ No newline at end of file +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java index 3ef55779a6bd..502152829968 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java @@ -56,7 +56,7 @@ public final class HttpAuthUtils { private static final String COOKIE_CLIENT_USER_NAME = "cu"; private static final String COOKIE_CLIENT_RAND_NUMBER = "rn"; private static final String COOKIE_KEY_VALUE_SEPARATOR = "="; - private final static Set COOKIE_ATTRIBUTES = + private static final Set COOKIE_ATTRIBUTES = new HashSet(Arrays.asList(COOKIE_CLIENT_USER_NAME, COOKIE_CLIENT_RAND_NUMBER)); /** @@ -93,10 +93,10 @@ public static String getKerberosServiceTicket(String principal, String host, */ public static String createCookieToken(String clientUserName) { StringBuffer sb = new StringBuffer(); - sb.append(COOKIE_CLIENT_USER_NAME).append(COOKIE_KEY_VALUE_SEPARATOR).append(clientUserName). - append(COOKIE_ATTR_SEPARATOR); - sb.append(COOKIE_CLIENT_RAND_NUMBER).append(COOKIE_KEY_VALUE_SEPARATOR). - append((new Random(System.currentTimeMillis())).nextLong()); + sb.append(COOKIE_CLIENT_USER_NAME).append(COOKIE_KEY_VALUE_SEPARATOR).append(clientUserName) + .append(COOKIE_ATTR_SEPARATOR); + sb.append(COOKIE_CLIENT_RAND_NUMBER).append(COOKIE_KEY_VALUE_SEPARATOR) + .append((new Random(System.currentTimeMillis())).nextLong()); return sb.toString(); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java index 11d26699fe78..52eb752f1e02 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java @@ -96,7 +96,7 @@ private static class CLIServiceProcessorFactory extends TProcessorFactory { private final ThriftCLIService service; private final Server saslServer; - public CLIServiceProcessorFactory(Server saslServer, ThriftCLIService service) { + CLIServiceProcessorFactory(Server saslServer, ThriftCLIService service) { super(null); this.service = service; this.saslServer = saslServer; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java index 479ebf32cec3..ab3ac6285aa0 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java @@ -25,10 +25,12 @@ * Possible values of SASL quality-of-protection value. */ public enum SaslQOP { - AUTH("auth"), // Authentication only. - AUTH_INT("auth-int"), // Authentication and integrity checking by using signatures. - AUTH_CONF("auth-conf"); // Authentication, integrity and confidentiality checking - // by using signatures and encryption. + // Authentication only. + AUTH("auth"), + // Authentication and integrity checking by using signatures. + AUTH_INT("auth-int"), + // Authentication, integrity and confidentiality checking by using signatures and encryption. + AUTH_CONF("auth-conf"); public final String saslQop; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java index a3af7b2d662b..791ddcbd2c5b 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java @@ -481,8 +481,8 @@ public synchronized String getDelegationTokenFromMetaStore(String owner) @Override public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, String owner, String renewer) throws HiveSQLException { - String delegationToken = sessionManager.getSession(sessionHandle). - getDelegationToken(authFactory, owner, renewer); + String delegationToken = sessionManager.getSession(sessionHandle) + .getDelegationToken(authFactory, owner, renewer); LOG.info(sessionHandle + ": getDelegationToken()"); return delegationToken; } @@ -490,8 +490,7 @@ public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory au @Override public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, String tokenStr) throws HiveSQLException { - sessionManager.getSession(sessionHandle). - cancelDelegationToken(authFactory, tokenStr); + sessionManager.getSession(sessionHandle).cancelDelegationToken(authFactory, tokenStr); LOG.info(sessionHandle + ": cancelDelegationToken()"); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java index 1334dde66375..86e57fbf31fe 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java @@ -111,7 +111,7 @@ public HiveSQLException(TStatus status) { /** * Converts current object to a {@link TStatus} object - * @return a {@link TStatus} object + * @return a {@link TStatus} object */ public TStatus toTStatus() { // TODO: convert sqlState, etc. @@ -125,8 +125,8 @@ public TStatus toTStatus() { /** * Converts the specified {@link Exception} object into a {@link TStatus} object - * @param e a {@link Exception} object - * @return a {@link TStatus} object + * @param e a {@link Exception} object + * @return a {@link TStatus} object */ public static TStatus toTStatus(Exception e) { if (e instanceof HiveSQLException) { @@ -155,7 +155,8 @@ private static List toString(Throwable cause, StackTraceElement[] parent if (parent != null) { int n = parent.length - 1; while (m >= 0 && n >= 0 && trace[m].equals(parent[n])) { - m--; n--; + m--; + n--; } } List detail = enroll(cause, trace, m); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java index 51ffb40369b2..116518011841 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java @@ -85,6 +85,7 @@ public static void validateTransition(OperationState oldState, if (OperationState.CLOSED.equals(newState)) { return; } + break; default: // fall-through } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java index a0ee2109dc54..7452137f077d 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java @@ -130,8 +130,8 @@ public void remove() { } private static class RemovableList extends ArrayList { - public RemovableList() { super(); } - public RemovableList(List rows) { super(rows); } + RemovableList() { super(); } + RemovableList(List rows) { super(rows); } @Override public void removeRange(int fromIndex, int toIndex) { super.removeRange(fromIndex, toIndex); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java index 87ac39b05186..05a6bf938404 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java @@ -42,7 +42,7 @@ public enum ClassicTableTypes { private final Map hiveToClientMap = new HashMap(); private final Map clientToHiveMap = new HashMap(); - public ClassicTableTypeMapping () { + public ClassicTableTypeMapping() { hiveToClientMap.put(TableType.MANAGED_TABLE.toString(), ClassicTableTypes.TABLE.toString()); hiveToClientMap.put(TableType.EXTERNAL_TABLE.toString(), diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java index 309f10f640f9..5efb0759383a 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java @@ -27,10 +27,8 @@ import java.util.Map.Entry; import java.util.regex.Pattern; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.ql.plan.HiveOperation; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java index 6df1e8a227f3..5273c386b83d 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java @@ -23,7 +23,6 @@ import java.util.Set; import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.ql.exec.FunctionInfo; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; @@ -103,7 +102,7 @@ public void runInternal() throws HiveSQLException { .getFunctionNames(CLIServiceUtils.patternToRegex(functionName)); for (String functionName : functionNames) { FunctionInfo functionInfo = FunctionRegistry.getFunctionInfo(functionName); - Object rowData[] = new Object[] { + Object[] rowData = new Object[] { null, // FUNCTION_CAT null, // FUNCTION_SCHEM functionInfo.getDisplayName(), // FUNCTION_NAME diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java index e56686abb7c5..d6f6280f1c39 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java @@ -18,16 +18,8 @@ package org.apache.hive.service.cli.operation; -import java.util.ArrayList; -import java.util.List; - import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; -import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hive.service.cli.FetchOrientation; import org.apache.hive.service.cli.HiveSQLException; import org.apache.hive.service.cli.OperationState; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java index a09b39a4e085..3ae012a72764 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java @@ -44,8 +44,8 @@ public class GetTableTypesOperation extends MetadataOperation { protected GetTableTypesOperation(HiveSession parentSession) { super(parentSession, OperationType.GET_TABLE_TYPES); - String tableMappingStr = getParentSession().getHiveConf(). - getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); + String tableMappingStr = getParentSession().getHiveConf() + .getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); tableTypeMapping = TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java index 0e2fdc657c4f..1a7ca79163d7 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java @@ -64,8 +64,8 @@ protected GetTablesOperation(HiveSession parentSession, this.catalogName = catalogName; this.schemaName = schemaName; this.tableName = tableName; - String tableMappingStr = getParentSession().getHiveConf(). - getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); + String tableMappingStr = getParentSession().getHiveConf() + .getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); tableTypeMapping = TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); if (tableTypes != null) { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java index 2a0fec27715d..0f72071d7e7d 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java @@ -35,7 +35,7 @@ */ public class GetTypeInfoOperation extends MetadataOperation { - private final static TableSchema RESULT_SET_SCHEMA = new TableSchema() + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() .addPrimitiveColumn("TYPE_NAME", Type.STRING_TYPE, "Type name") .addPrimitiveColumn("DATA_TYPE", Type.INT_TYPE, diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java index 70340bd13cbc..cb804318ace9 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java @@ -60,15 +60,15 @@ private static class NameFilter extends Filter { /* Patterns that are excluded in verbose logging level. * Filter out messages coming from log processing classes, or we'll run an infinite loop. */ - private static final Pattern verboseExcludeNamePattern = Pattern.compile(Joiner.on("|"). - join(new String[] {LOG.getName(), OperationLog.class.getName(), + private static final Pattern verboseExcludeNamePattern = Pattern.compile(Joiner.on("|") + .join(new String[] {LOG.getName(), OperationLog.class.getName(), OperationManager.class.getName()})); /* Patterns that are included in execution logging level. * In execution mode, show only select logger messages. */ - private static final Pattern executionIncludeNamePattern = Pattern.compile(Joiner.on("|"). - join(new String[] {"org.apache.hadoop.mapreduce.JobSubmitter", + private static final Pattern executionIncludeNamePattern = Pattern.compile(Joiner.on("|") + .join(new String[] {"org.apache.hadoop.mapreduce.JobSubmitter", "org.apache.hadoop.mapreduce.Job", "SessionState", Task.class.getName(), "org.apache.hadoop.hive.ql.exec.spark.status.SparkJobMonitor"})); @@ -88,7 +88,7 @@ private void setCurrentNamePattern(OperationLog.LoggingLevel mode) { } } - public NameFilter( + NameFilter( OperationLog.LoggingLevel loggingMode, OperationManager op) { this.operationManager = op; this.loggingMode = loggingMode; @@ -131,7 +131,7 @@ public int decide(LoggingEvent ev) { /** This is where the log message will go to */ private final CharArrayWriter writer = new CharArrayWriter(); - private void setLayout (boolean isVerbose, Layout lo) { + private void setLayout(boolean isVerbose, Layout lo) { if (isVerbose) { if (lo == null) { lo = CLIServiceUtils.verboseLayout; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java index 4595ef56fcee..6c819876a556 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java @@ -18,7 +18,6 @@ package org.apache.hive.service.cli.operation; -import java.util.ArrayList; import java.util.List; import org.apache.hadoop.hive.conf.HiveConf; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java index 33ee16b80beb..a35405484a76 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java @@ -326,7 +326,7 @@ public TableSchema getResultSetSchema() throws HiveSQLException { return resultSchema; } - private transient final List convey = new ArrayList(); + private final transient List convey = new ArrayList(); @Override public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java index 3a8a07f44f20..e392c459cf58 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java @@ -27,18 +27,18 @@ public interface TableTypeMapping { * @param clientTypeName * @return */ - public String mapToHiveType (String clientTypeName); + String mapToHiveType(String clientTypeName); /** * Map hive's table type name to client's table type * @param clientTypeName * @return */ - public String mapToClientType (String hiveTypeName); + String mapToClientType(String hiveTypeName); /** * Get all the table types of this mapping * @return */ - public Set getTableTypeNames(); + Set getTableTypeNames(); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java index 9b04d679df1c..b72c18b2b213 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java @@ -18,8 +18,6 @@ package org.apache.hive.service.cli.session; -import java.util.Map; - import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hive.service.cli.SessionHandle; @@ -27,7 +25,6 @@ import org.apache.hive.service.cli.thrift.TProtocolVersion; import java.io.File; -import java.util.Map; /** * Methods that don't need to be executed under a doAs diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java deleted file mode 100644 index 06388cc795b9..000000000000 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java +++ /dev/null @@ -1,37 +0,0 @@ -/** - * 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.hive.service.cli.session; - -import org.apache.hadoop.hive.ql.hooks.Hook; -import org.apache.hive.service.cli.HiveSQLException; - -/** - * HiveSessionHook. - * HiveServer2 session level Hook interface. The run method is executed - * when session manager starts a new session - * - */ -public interface HiveSessionHook extends Hook { - - /** - * @param sessionHookContext context - * @throws HiveSQLException - */ - public void run(HiveSessionHookContext sessionHookContext) throws HiveSQLException; -} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java index 156c8147f9d9..0a10dba8b479 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java @@ -30,17 +30,17 @@ public interface HiveSessionHookContext { * Retrieve session conf * @return */ - public HiveConf getSessionConf(); + HiveConf getSessionConf(); /** * The get the username starting the session * @return */ - public String getSessionUser(); + String getSessionUser(); /** * Retrieve handle for the session * @return */ - public String getSessionHandle(); + String getSessionHandle(); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java index a29e5d1d81c1..762dbb2faade 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.hive.shims.Utils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hive.service.auth.HiveAuthFactory; @@ -83,7 +82,7 @@ public UserGroupInformation getSessionUgi() { return this.sessionUgi; } - public String getDelegationToken () { + public String getDelegationToken() { return this.delegationTokenStr; } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java index 5b10521febfc..8e539512f741 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java @@ -57,7 +57,7 @@ public Object invoke(Object arg0, final Method method, final Object[] args) return invoke(method, args); } return ugi.doAs( - new PrivilegedExceptionAction () { + new PrivilegedExceptionAction() { @Override public Object run() throws HiveSQLException { return invoke(method, args); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java index e31570bdfba5..de066dd406c7 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Date; -import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Future; @@ -35,7 +34,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.ql.hooks.HookUtils; import org.apache.hive.service.CompositeService; import org.apache.hive.service.cli.HiveSQLException; import org.apache.hive.service.cli.SessionHandle; @@ -151,7 +149,7 @@ public synchronized void start() { } private void startTimeoutChecker() { - final long interval = Math.max(checkInterval, 3000l); // minimum 3 seconds + final long interval = Math.max(checkInterval, 3000L); // minimum 3 seconds Runnable timeoutChecker = new Runnable() { @Override public void run() { @@ -268,17 +266,6 @@ public SessionHandle openSession(TProtocolVersion protocol, String username, Str if (isOperationLogEnabled) { session.setOperationLogSessionDir(operationLogRootDir); } - try { - executeSessionHooks(session); - } catch (Exception e) { - try { - session.close(); - } catch (Throwable t) { - LOG.warn("Error closing session", t); - } - session = null; - throw new HiveSQLException("Failed to execute session hooks", e); - } handleToSession.put(session.getSessionHandle(), session); return session.getSessionHandle(); } @@ -361,15 +348,6 @@ public static void clearProxyUserName() { threadLocalProxyUserName.remove(); } - // execute session hooks - private void executeSessionHooks(HiveSession session) throws Exception { - List sessionHooks = HookUtils.getHooks(hiveConf, - HiveConf.ConfVars.HIVE_SERVER2_SESSION_HOOK, HiveSessionHook.class); - for (HiveSessionHook sessionHook : sessionHooks) { - sessionHook.run(new HiveSessionHookContextImpl(session)); - } - } - public Future submitBackgroundOperation(Runnable r) { return backgroundOperationPool.submit(r); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 5a0f1c83c70f..ad7a9a238f8a 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -18,6 +18,7 @@ package org.apache.hive.service.cli.thrift; +import javax.security.auth.login.LoginException; import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; @@ -25,8 +26,6 @@ import java.util.Map; import java.util.concurrent.TimeUnit; -import javax.security.auth.login.LoginException; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; @@ -36,17 +35,7 @@ import org.apache.hive.service.ServiceUtils; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.auth.TSetIpAddressProcessor; -import org.apache.hive.service.cli.CLIService; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.FetchType; -import org.apache.hive.service.cli.GetInfoType; -import org.apache.hive.service.cli.GetInfoValue; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationHandle; -import org.apache.hive.service.cli.OperationStatus; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.SessionHandle; -import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.*; import org.apache.hive.service.cli.session.SessionManager; import org.apache.hive.service.server.HiveServer2; import org.apache.thrift.TException; @@ -223,23 +212,7 @@ public InetAddress getServerIPAddress() { public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) throws TException { TGetDelegationTokenResp resp = new TGetDelegationTokenResp(); - - if (hiveAuthFactory == null) { - resp.setStatus(unsecureTokenErrorStatus()); - } else { - try { - String token = cliService.getDelegationToken( - new SessionHandle(req.getSessionHandle()), - hiveAuthFactory, req.getOwner(), req.getRenewer()); - resp.setDelegationToken(token); - resp.setStatus(OK_STATUS); - } catch (HiveSQLException e) { - LOG.error("Error obtaining delegation token", e); - TStatus tokenErrorStatus = HiveSQLException.toTStatus(e); - tokenErrorStatus.setSqlState("42000"); - resp.setStatus(tokenErrorStatus); - } - } + resp.setStatus(notSupportTokenErrorStatus()); return resp; } @@ -247,19 +220,7 @@ public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) throws TException { TCancelDelegationTokenResp resp = new TCancelDelegationTokenResp(); - - if (hiveAuthFactory == null) { - resp.setStatus(unsecureTokenErrorStatus()); - } else { - try { - cliService.cancelDelegationToken(new SessionHandle(req.getSessionHandle()), - hiveAuthFactory, req.getDelegationToken()); - resp.setStatus(OK_STATUS); - } catch (HiveSQLException e) { - LOG.error("Error canceling delegation token", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - } + resp.setStatus(notSupportTokenErrorStatus()); return resp; } @@ -267,25 +228,13 @@ public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenRe public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) throws TException { TRenewDelegationTokenResp resp = new TRenewDelegationTokenResp(); - if (hiveAuthFactory == null) { - resp.setStatus(unsecureTokenErrorStatus()); - } else { - try { - cliService.renewDelegationToken(new SessionHandle(req.getSessionHandle()), - hiveAuthFactory, req.getDelegationToken()); - resp.setStatus(OK_STATUS); - } catch (HiveSQLException e) { - LOG.error("Error obtaining renewing token", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - } + resp.setStatus(notSupportTokenErrorStatus()); return resp; } - private TStatus unsecureTokenErrorStatus() { + private TStatus notSupportTokenErrorStatus() { TStatus errorStatus = new TStatus(TStatusCode.ERROR_STATUS); - errorStatus.setErrorMessage("Delegation token only supported over remote " + - "client with kerberos authentication"); + errorStatus.setErrorMessage("Delegation token is not supported"); return errorStatus; } @@ -722,8 +671,8 @@ private String getProxyUser(String realUser, Map sessionConf, } // If there's no authentication, then directly substitute the user - if (HiveAuthFactory.AuthTypes.NONE.toString(). - equalsIgnoreCase(hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION))) { + if (HiveAuthFactory.AuthTypes.NONE.toString() + .equalsIgnoreCase(hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION))) { return proxyUser; } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index 3b57efa38b58..37e4845cceb9 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -37,8 +37,7 @@ import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.protocol.TProtocolFactory; import org.apache.thrift.server.TServlet; -import org.eclipse.jetty.server.nio.SelectChannelConnector; -import org.eclipse.jetty.server.ssl.SslSelectChannelConnector; +import org.eclipse.jetty.server.ServerConnector; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.util.ssl.SslContextFactory; @@ -59,9 +58,6 @@ public ThriftHttpCLIService(CLIService cliService) { @Override public void run() { try { - // HTTP Server - httpServer = new org.eclipse.jetty.server.Server(); - // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests String threadPoolName = "HiveServer2-HttpHandler-Pool"; @@ -69,10 +65,12 @@ public void run() { workerKeepAliveTime, TimeUnit.SECONDS, new SynchronousQueue(), new ThreadFactoryWithGarbageCleanup(threadPoolName)); ExecutorThreadPool threadPool = new ExecutorThreadPool(executorService); - httpServer.setThreadPool(threadPool); + + // HTTP Server + httpServer = new org.eclipse.jetty.server.Server(threadPool); // Connector configs - SelectChannelConnector connector = new SelectChannelConnector(); + ServerConnector connector = new ServerConnector(httpServer); boolean useSsl = hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL); String schemeName = useSsl ? "https" : "http"; // Change connector if SSL is used @@ -92,14 +90,14 @@ public void run() { Arrays.toString(sslContextFactory.getExcludeProtocols())); sslContextFactory.setKeyStorePath(keyStorePath); sslContextFactory.setKeyStorePassword(keyStorePassword); - connector = new SslSelectChannelConnector(sslContextFactory); + connector = new ServerConnector(httpServer, sslContextFactory); } connector.setPort(portNum); // Linux:yes, Windows:no connector.setReuseAddress(!Shell.WINDOWS); int maxIdleTime = (int) hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME, TimeUnit.MILLISECONDS); - connector.setMaxIdleTime(maxIdleTime); + connector.setIdleTimeout(maxIdleTime); httpServer.addConnector(connector); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java index 56c8cb6e5459..e15d2d0566d2 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.io.UnsupportedEncodingException; import java.security.PrivilegedExceptionAction; -import java.util.Arrays; import java.util.Map; import java.util.Random; import java.util.Set; @@ -241,9 +240,9 @@ private String getClientNameFromCookie(Cookie[] cookies) { * Each cookie is of the format [key]=[value] */ private String toCookieStr(Cookie[] cookies) { - String cookieStr = ""; + String cookieStr = ""; - for (Cookie c : cookies) { + for (Cookie c : cookies) { cookieStr += c.getName() + "=" + c.getValue() + " ;\n"; } return cookieStr; @@ -458,7 +457,7 @@ private String getPrincipalWithoutRealmAndHost(String fullPrincipal) private String getUsername(HttpServletRequest request, String authType) throws HttpAuthenticationException { - String creds[] = getAuthHeaderTokens(request, authType); + String[] creds = getAuthHeaderTokens(request, authType); // Username must be present if (creds[0] == null || creds[0].isEmpty()) { throw new HttpAuthenticationException("Authorization header received " + @@ -469,7 +468,7 @@ private String getUsername(HttpServletRequest request, String authType) private String getPassword(HttpServletRequest request, String authType) throws HttpAuthenticationException { - String creds[] = getAuthHeaderTokens(request, authType); + String[] creds = getAuthHeaderTokens(request, authType); // Password must be present if (creds[1] == null || creds[1].isEmpty()) { throw new HttpAuthenticationException("Authorization header received " + diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java index 1500e537cef5..9bf96cff572e 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java @@ -236,8 +236,8 @@ ServerOptionsExecutor getServerOptionsExecutor() { /** * The executor interface for running the appropriate HiveServer2 command based on parsed options */ - static interface ServerOptionsExecutor { - public void execute(); + interface ServerOptionsExecutor { + void execute(); } /** diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 94b1ced9908d..cef5912c6240 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -26,7 +26,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService} +import org.apache.hive.service.cli.thrift.ThriftHttpCLIService import org.apache.hive.service.server.HiveServer2 import org.apache.spark.SparkContext @@ -34,7 +34,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerJobStart} import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.hive.{HiveSharedState, HiveUtils} +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab import org.apache.spark.sql.internal.SQLConf @@ -271,7 +271,7 @@ object HiveThriftServer2 extends Logging { private[hive] class HiveThriftServer2(sqlContext: SQLContext) extends HiveServer2 - with ReflectedCompositeService { + with ReflectedCompositeService with Logging { // state is tracked internally so that the server only attempts to shut down if it successfully // started, and then once only. private val started = new AtomicBoolean(false) @@ -281,20 +281,18 @@ private[hive] class HiveThriftServer2(sqlContext: SQLContext) setSuperField(this, "cliService", sparkSqlCliService) addService(sparkSqlCliService) - val thriftCliService = if (isHTTPTransportMode(hiveConf)) { - new ThriftHttpCLIService(sparkSqlCliService) - } else { - new ThriftBinaryCLIService(sparkSqlCliService) + if (isBinaryTransportMode(hiveConf)) { + logWarning("Binary mode is not supported, use HTTP mode instead") } - + val thriftCliService = new ThriftHttpCLIService(sparkSqlCliService) setSuperField(this, "thriftCLIService", thriftCliService) addService(thriftCliService) initCompositeService(hiveConf) } - private def isHTTPTransportMode(hiveConf: HiveConf): Boolean = { + private def isBinaryTransportMode(hiveConf: HiveConf): Boolean = { val transportMode = hiveConf.getVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE) - transportMode.toLowerCase(Locale.ENGLISH).equals("http") + transportMode.toLowerCase(Locale.ENGLISH).equals("binary") } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 665a44e51a0c..638911599aad 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -54,13 +54,15 @@ private[hive] object SparkSQLEnv extends Logging { "spark.kryo.referenceTracking", maybeKryoReferenceTracking.getOrElse("false")) - sparkContext = new SparkContext(sparkConf) - sqlContext = SparkSession.withHiveSupport(sparkContext).wrapped - val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState] + val sparkSession = SparkSession.builder.config(sparkConf).enableHiveSupport().getOrCreate() + sparkContext = sparkSession.sparkContext + sqlContext = sparkSession.sqlContext + + val sessionState = sparkSession.sessionState.asInstanceOf[HiveSessionState] sessionState.metadataHive.setOut(new PrintStream(System.out, true, "UTF-8")) sessionState.metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8")) sessionState.metadataHive.setError(new PrintStream(System.err, true, "UTF-8")) - sqlContext.setConf("spark.sql.hive.version", HiveUtils.hiveExecutionVersion) + sparkSession.conf.set("spark.sql.hive.version", HiveUtils.hiveExecutionVersion) } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index 55a93ea06ba5..b3f4944c9129 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -55,8 +55,8 @@ object TestData { val smallKvWithNull = getTestDataFilePath("small_kv_with_null.txt") } -class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { - override def mode: ServerMode.Value = ServerMode.binary +class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { + override def mode: ServerMode.Value = ServerMode.http private def withCLIServiceClient(f: ThriftCLIServiceClient => Unit): Unit = { // Transport creation logic below mimics HiveConnection.createBinaryTransport @@ -70,7 +70,8 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { try f(client) finally transport.close() } - test("GetInfo Thrift API") { + // TODO: update this test to work in HTTP mode + ignore("GetInfo Thrift API") { withCLIServiceClient { client => val user = System.getProperty("user.name") val sessionHandle = client.openSession(user, "") @@ -533,7 +534,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } assert(rs1.next()) - assert(rs1.getString(1) === "Usage: To be added.") + assert(rs1.getString(1) === "Usage: N/A.") val dataPath = "../hive/src/test/resources/data/files/kv1.txt" @@ -566,7 +567,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } class SingleSessionSuite extends HiveThriftJdbcTest { - override def mode: ServerMode.Value = ServerMode.binary + override def mode: ServerMode.Value = ServerMode.http override protected def extraConf: Seq[String] = "--conf spark.sql.hive.thriftServer.singleSession=true" :: Nil @@ -607,7 +608,7 @@ class SingleSessionSuite extends HiveThriftJdbcTest { } assert(rs2.next()) - assert(rs2.getString(1) === "Usage: To be added.") + assert(rs2.getString(1) === "Usage: N/A.") } finally { statement.executeQuery("DROP TEMPORARY FUNCTION udtf_count2") } @@ -616,38 +617,6 @@ class SingleSessionSuite extends HiveThriftJdbcTest { } } -class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { - override def mode: ServerMode.Value = ServerMode.http - - test("JDBC query execution") { - withJdbcStatement { statement => - val queries = Seq( - "SET spark.sql.shuffle.partitions=3", - "DROP TABLE IF EXISTS test", - "CREATE TABLE test(key INT, val STRING)", - s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test", - "CACHE TABLE test") - - queries.foreach(statement.execute) - - assertResult(5, "Row count mismatch") { - val resultSet = statement.executeQuery("SELECT COUNT(*) FROM test") - resultSet.next() - resultSet.getInt(1) - } - } - } - - test("Checks Hive version") { - withJdbcStatement { statement => - val resultSet = statement.executeQuery("SET spark.sql.hive.version") - resultSet.next() - assert(resultSet.getString(1) === "spark.sql.hive.version") - assert(resultSet.getString(2) === HiveUtils.hiveExecutionVersion) - } - } -} - object ServerMode extends Enumeration { val binary, http = Value } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala index bf431cd6b026..b6b9de1ba663 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala @@ -36,7 +36,7 @@ class UISeleniumSuite implicit var webDriver: WebDriver = _ var server: HiveThriftServer2 = _ val uiPort = 20000 + Random.nextInt(10000) - override def mode: ServerMode.Value = ServerMode.binary + override def mode: ServerMode.Value = ServerMode.http override def beforeAll(): Unit = { webDriver = new HtmlUnitDriver { diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index f082035852cc..54fb440b3374 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -503,7 +503,15 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // We have converted the useful parts of these tests to tests // in org.apache.spark.sql.hive.execution.SQLQuerySuite. "drop_database_removes_partition_dirs", - "drop_table_removes_partition_dirs" + "drop_table_removes_partition_dirs", + + // These tests use EXPLAIN FORMATTED, which is not supported + "input4", + "join0", + "plan_json", + + // This test uses CREATE EXTERNAL TABLE without specifying LOCATION + "alter2" ) /** @@ -516,7 +524,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "add_partition_no_whitelist", "add_partition_with_whitelist", "alias_casted_column", - "alter2", "alter_partition_with_whitelist", "alter_rename_partition", "ambiguous_col", @@ -699,7 +706,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "input26", "input28", "input2_limit", - "input4", "input40", "input41", "input49", @@ -728,7 +734,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "insert1_overwrite_partitions", "insert2_overwrite_partitions", "insert_compressed", - "join0", "join1", "join10", "join11", @@ -866,7 +871,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "partition_type_check", "partition_varchar1", "partition_wise_fileformat9", - "plan_json", "ppd1", "ppd2", "ppd_clusterby", @@ -924,6 +928,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "stats_aggregator_error_1", "stats_publisher_error_1", "subq2", + "subquery_exists", + "subquery_exists_having", + "subquery_notexists", + "subquery_notexists_having", + "subquery_in", + "subquery_in_having", + "subquery_notin_having", "tablename_with_select", "timestamp_3", "timestamp_comparison", diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 177b6884fa13..c8b20f0afc4e 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -60,7 +60,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} @@ -77,6 +77,10 @@ org.eclipse.jetty jetty-servlet + + org.eclipse.jetty + jetty-servlets + @@ -93,6 +97,11 @@ selenium-java test + + org.seleniumhq.selenium + selenium-htmlunit-driver + test + org.mockito mockito-core diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 7d8b8679c594..6ececb1062ae 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -84,7 +84,7 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) assert(framework != null, "Checkpoint.framework is null") assert(graph != null, "Checkpoint.graph is null") assert(checkpointTime != null, "Checkpoint.checkpointTime is null") - logInfo("Checkpoint for time " + checkpointTime + " validated") + logInfo(s"Checkpoint for time $checkpointTime validated") } } @@ -103,7 +103,10 @@ object Checkpoint extends Logging { new Path(checkpointDir, PREFIX + checkpointTime.milliseconds + ".bk") } - /** Get checkpoint files present in the give directory, ordered by oldest-first */ + /** + * @param checkpointDir checkpoint directory to read checkpoint files from + * @return checkpoint files from the `checkpointDir` checkpoint directory, ordered by oldest-first + */ def getCheckpointFiles(checkpointDir: String, fsOption: Option[FileSystem] = None): Seq[Path] = { def sortFunc(path1: Path, path2: Path): Boolean = { @@ -121,11 +124,11 @@ object Checkpoint extends Logging { val filtered = paths.filter(p => REGEX.findFirstIn(p.toString).nonEmpty) filtered.sortWith(sortFunc) } else { - logWarning("Listing " + path + " returned null") + logWarning(s"Listing $path returned null") Seq.empty } } else { - logInfo("Checkpoint directory " + path + " does not exist") + logWarning(s"Checkpoint directory $path does not exist") Seq.empty } } @@ -205,7 +208,7 @@ class CheckpointWriter( // time of a batch is greater than the batch interval, checkpointing for completing an old // batch may run after checkpointing of a new batch. If this happens, checkpoint of an old // batch actually has the latest information, so we want to recovery from it. Therefore, we - // also use the latest checkpoint time as the file name, so that we can recovery from the + // also use the latest checkpoint time as the file name, so that we can recover from the // latest checkpoint file. // // Note: there is only one thread writing the checkpoint files, so we don't need to worry @@ -216,8 +219,7 @@ class CheckpointWriter( while (attempts < MAX_ATTEMPTS && !stopped) { attempts += 1 try { - logInfo("Saving checkpoint for time " + checkpointTime + " to file '" + checkpointFile - + "'") + logInfo(s"Saving checkpoint for time $checkpointTime to file '$checkpointFile'") // Write checkpoint to temp file if (fs.exists(tempFile)) { @@ -237,39 +239,38 @@ class CheckpointWriter( fs.delete(backupFile, true) // just in case it exists } if (!fs.rename(checkpointFile, backupFile)) { - logWarning("Could not rename " + checkpointFile + " to " + backupFile) + logWarning(s"Could not rename $checkpointFile to $backupFile") } } // Rename temp file to the final checkpoint file if (!fs.rename(tempFile, checkpointFile)) { - logWarning("Could not rename " + tempFile + " to " + checkpointFile) + logWarning(s"Could not rename $tempFile to $checkpointFile") } // Delete old checkpoint files val allCheckpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, Some(fs)) if (allCheckpointFiles.size > 10) { allCheckpointFiles.take(allCheckpointFiles.size - 10).foreach { file => - logInfo("Deleting " + file) + logInfo(s"Deleting $file") fs.delete(file, true) } } // All done, print success val finishTime = System.currentTimeMillis() - logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + checkpointFile + - "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " ms") + logInfo(s"Checkpoint for time $checkpointTime saved to file '$checkpointFile'" + + s", took ${bytes.length} bytes and ${finishTime - startTime} ms") jobGenerator.onCheckpointCompletion(checkpointTime, clearCheckpointDataLater) return } catch { case ioe: IOException => - logWarning("Error in attempt " + attempts + " of writing checkpoint to " - + checkpointFile, ioe) + val msg = s"Error in attempt $attempts of writing checkpoint to '$checkpointFile'" + logWarning(msg, ioe) fs = null } } - logWarning("Could not write checkpoint for time " + checkpointTime + " to file " - + checkpointFile + "'") + logWarning(s"Could not write checkpoint for time $checkpointTime to file '$checkpointFile'") } } @@ -278,7 +279,7 @@ class CheckpointWriter( val bytes = Checkpoint.serialize(checkpoint, conf) executor.execute(new CheckpointWriteHandler( checkpoint.checkpointTime, bytes, clearCheckpointDataLater)) - logInfo("Submitted checkpoint of time " + checkpoint.checkpointTime + " writer queue") + logInfo(s"Submitted checkpoint of time ${checkpoint.checkpointTime} to writer queue") } catch { case rej: RejectedExecutionException => logError("Could not submit checkpoint task to the thread pool executor", rej) @@ -295,8 +296,8 @@ class CheckpointWriter( executor.shutdownNow() } val endTime = System.currentTimeMillis() - logInfo("CheckpointWriter executor terminated ? " + terminated + - ", waited for " + (endTime - startTime) + " ms.") + logInfo(s"CheckpointWriter executor terminated? $terminated," + + s" waited for ${endTime - startTime} ms.") stopped = true } } @@ -336,20 +337,20 @@ object CheckpointReader extends Logging { } // Try to read the checkpoint files in the order - logInfo("Checkpoint files found: " + checkpointFiles.mkString(",")) + logInfo(s"Checkpoint files found: ${checkpointFiles.mkString(",")}") var readError: Exception = null checkpointFiles.foreach { file => - logInfo("Attempting to load checkpoint from file " + file) + logInfo(s"Attempting to load checkpoint from file $file") try { val fis = fs.open(file) val cp = Checkpoint.deserialize(fis, conf) - logInfo("Checkpoint successfully loaded from file " + file) - logInfo("Checkpoint was generated at time " + cp.checkpointTime) + logInfo(s"Checkpoint successfully loaded from file $file") + logInfo(s"Checkpoint was generated at time ${cp.checkpointTime}") return Some(cp) } catch { case e: Exception => readError = e - logWarning("Error reading checkpoint from file " + file, e) + logWarning(s"Error reading checkpoint from file $file", e) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index 43632f37ccb1..a0a40fcee26d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -240,7 +240,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * This is more efficient than reduceByWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse reduce function + * @param invReduceFunc inverse reduce function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index 2a80cf446658..dec983165fb3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -336,7 +336,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * However, it is applicable to only "invertible reduce functions". * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse function + * @param invReduceFunc inverse function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 583f5a48d1a6..01dcfcf24b0f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -793,7 +793,8 @@ abstract class DStream[T: ClassTag] ( * This is more efficient than reduceByWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse reduce function + * @param invReduceFunc inverse reduce function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 36f50e04db42..ed9305875cb7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -195,10 +195,16 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( ) logDebug(s"Getting new files for time $currentTime, " + s"ignoring files older than $modTimeIgnoreThreshold") - val filter = new PathFilter { + + val newFileFilter = new PathFilter { def accept(path: Path): Boolean = isNewFile(path, currentTime, modTimeIgnoreThreshold) } - val newFiles = fs.listStatus(directoryPath, filter).map(_.getPath.toString) + val directoryFilter = new PathFilter { + override def accept(path: Path): Boolean = fs.getFileStatus(path).isDirectory + } + val directories = fs.globStatus(directoryPath, directoryFilter).map(_.getPath) + val newFiles = directories.flatMap(dir => + fs.listStatus(dir, newFileFilter).map(_.getPath.toString)) val timeTaken = clock.getTimeMillis() - lastNewFileFindingTime logInfo("Finding new files took " + timeTaken + " ms") logDebug("# cached file times = " + fileToModTime.size) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index b6394e36b515..2f2a6d13dd79 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -290,7 +290,8 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) * However, it is applicable to only "invertible reduce functions". * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse reduce function + * @param invReduceFunc inverse reduce function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index a2653000af55..00d506c2f18b 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -140,10 +140,10 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } test("binary records stream") { - val testDir: File = null + var testDir: File = null try { val batchDuration = Seconds(2) - val testDir = Utils.createTempDir() + testDir = Utils.createTempDir() // Create a file that exists before the StreamingContext is created: val existingFile = new File(testDir, "0") Files.write("0\n", existingFile, StandardCharsets.UTF_8) @@ -198,6 +198,68 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { testFileStream(newFilesOnly = false) } + test("file input stream - wildcard") { + var testDir: File = null + try { + val batchDuration = Seconds(2) + testDir = Utils.createTempDir() + val testSubDir1 = Utils.createDirectory(testDir.toString, "tmp1") + val testSubDir2 = Utils.createDirectory(testDir.toString, "tmp2") + + // Create a file that exists before the StreamingContext is created: + val existingFile = new File(testDir, "0") + Files.write("0\n", existingFile, StandardCharsets.UTF_8) + assert(existingFile.setLastModified(10000) && existingFile.lastModified === 10000) + + val pathWithWildCard = testDir.toString + "/*/" + + // Set up the streaming context and input streams + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + clock.setTime(existingFile.lastModified + batchDuration.milliseconds) + val batchCounter = new BatchCounter(ssc) + // monitor "testDir/*/" + val fileStream = ssc.fileStream[LongWritable, Text, TextInputFormat]( + pathWithWildCard).map(_._2.toString) + val outputQueue = new ConcurrentLinkedQueue[Seq[String]] + val outputStream = new TestOutputStream(fileStream, outputQueue) + outputStream.register() + ssc.start() + + // Advance the clock so that the files are created after StreamingContext starts, but + // not enough to trigger a batch + clock.advance(batchDuration.milliseconds / 2) + + def createFileAndAdvenceTime(data: Int, dir: File): Unit = { + val file = new File(testSubDir1, data.toString) + Files.write(data + "\n", file, StandardCharsets.UTF_8) + assert(file.setLastModified(clock.getTimeMillis())) + assert(file.lastModified === clock.getTimeMillis()) + logInfo("Created file " + file) + // Advance the clock after creating the file to avoid a race when + // setting its modification time + clock.advance(batchDuration.milliseconds) + eventually(eventuallyTimeout) { + assert(batchCounter.getNumCompletedBatches === data) + } + } + // Over time, create files in the temp directory 1 + val input1 = Seq(1, 2, 3, 4, 5) + input1.foreach(i => createFileAndAdvenceTime(i, testSubDir1)) + + // Over time, create files in the temp directory 1 + val input2 = Seq(6, 7, 8, 9, 10) + input2.foreach(i => createFileAndAdvenceTime(i, testSubDir2)) + + // Verify that all the files have been read + val expectedOutput = (input1 ++ input2).map(_.toString).toSet + assert(outputQueue.asScala.flatten.toSet === expectedOutput) + } + } finally { + if (testDir != null) Utils.deleteRecursively(testDir) + } + } + test("multi-thread receiver") { // set up the test receiver val numThreads = 10 @@ -363,10 +425,10 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } def testFileStream(newFilesOnly: Boolean) { - val testDir: File = null + var testDir: File = null try { val batchDuration = Seconds(2) - val testDir = Utils.createTempDir() + testDir = Utils.createTempDir() // Create a file that exists before the StreamingContext is created: val existingFile = new File(testDir, "0") Files.write("0\n", existingFile, StandardCharsets.UTF_8) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 4be4882938df..e97427991bf9 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -29,6 +29,7 @@ import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.concurrent.Eventually._ import org.apache.spark._ +import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.internal.Logging import org.apache.spark.memory.StaticMemoryManager import org.apache.spark.network.netty.NettyBlockTransferService @@ -57,7 +58,8 @@ class ReceivedBlockHandlerSuite val hadoopConf = new Configuration() val streamId = 1 val securityMgr = new SecurityManager(conf) - val mapOutputTracker = new MapOutputTrackerMaster(conf) + val broadcastManager = new BroadcastManager(true, conf, securityMgr) + val mapOutputTracker = new MapOutputTrackerMaster(conf, broadcastManager, true) val shuffleManager = new SortShuffleManager(conf) val serializer = new KryoSerializer(conf) var serializerManager = new SerializerManager(serializer, conf) diff --git a/yarn/pom.xml b/yarn/pom.xml index 328bb6678db9..e07b93ab9545 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -30,6 +30,7 @@ Spark Project YARN yarn + 1.9 @@ -53,7 +54,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} org.apache.hadoop @@ -101,6 +102,10 @@ org.eclipse.jetty jetty-servlet + + org.eclipse.jetty + jetty-servlets + com.sun.jersey jersey-core test + ${jersey-1.version} com.sun.jersey jersey-json test + ${jersey-1.version} com.sun.jersey jersey-server test + ${jersey-1.version} + + + com.sun.jersey.contribs + jersey-guice + test + ${jersey-1.version}
    SourceArtifact
    Kafka spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}
    Kafka spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}
    Flume spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}
    Kinesis
    spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}} [Amazon Software License]